diff --git a/.bazelrc b/.bazelrc index 29f44012e841..46e9b0725c1c 100644 --- a/.bazelrc +++ b/.bazelrc @@ -43,6 +43,8 @@ build:devdarwinx86_64 --platforms=//build/toolchains:darwin_x86_64 build:devdarwinx86_64 --config=dev build:dev --define cockroach_bazel_dev=y build:dev --stamp --workspace_status_command=./build/bazelutil/stamp.sh +build:dev --action_env=PATH +build:dev --host_action_env=PATH build:nonogo --define cockroach_nonogo=y # vi: ft=sh diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 0e606e9fd599..3f92e4f1f473 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -118,6 +118,7 @@ ALL_TESTS = [ "//pkg/keys:keys_test", "//pkg/kv/bulk:bulk_test", "//pkg/kv/kvclient/kvcoord:kvcoord_test", + "//pkg/kv/kvclient/kvstreamer:kvstreamer_test", "//pkg/kv/kvclient/rangecache:rangecache_test", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer:rangefeedbuffer_test", "//pkg/kv/kvclient/rangefeed:rangefeed_test", diff --git a/pkg/kv/kvclient/kvcoord/batch.go b/pkg/kv/kvclient/kvcoord/batch.go index 2f741385c34d..7da549d933c0 100644 --- a/pkg/kv/kvclient/kvcoord/batch.go +++ b/pkg/kv/kvclient/kvcoord/batch.go @@ -18,7 +18,7 @@ import ( var emptySpan = roachpb.Span{} -// truncate restricts all requests to the given key range and returns new, +// Truncate restricts all requests to the given key range and returns new, // truncated, requests. All returned requests are "truncated" to the given span, // and requests which are found to not overlap the given span at all are // removed. A mapping of response index to request index is returned. For @@ -27,8 +27,8 @@ var emptySpan = roachpb.Span{} // reqs = Put[a], Put[c], Put[b], // rs = [a,bb], // -// then truncate(reqs,rs) returns (Put[a], Put[b]) and positions [0,2]. -func truncate( +// then Truncate(reqs,rs) returns (Put[a], Put[b]) and positions [0,2]. +func Truncate( reqs []roachpb.RequestUnion, rs roachpb.RSpan, ) ([]roachpb.RequestUnion, []int, error) { truncateOne := func(args roachpb.Request) (bool, roachpb.Span, error) { @@ -191,18 +191,18 @@ func prev(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { return candidate, nil } -// next gives the left boundary of the union of all requests which don't affect +// Next gives the left boundary of the union of all requests which don't affect // keys less than the given key. Note that the left boundary is inclusive, that // is, the returned RKey is the inclusive left endpoint of the keys the request // should operate on next. // -// Informally, a call `next(reqs, k)` means: we've already executed the parts of +// Informally, a call `Next(reqs, k)` means: we've already executed the parts of // `reqs` that intersect `[KeyMin, k)`; please tell me how far to the right the // next relevant request begins. // // TODO(tschottdorf): again, better on BatchRequest itself, but can't pull // 'keys' into 'proto'. -func next(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { +func Next(reqs []roachpb.RequestUnion, k roachpb.RKey) (roachpb.RKey, error) { candidate := roachpb.RKeyMax for _, union := range reqs { inner := union.GetInner() diff --git a/pkg/kv/kvclient/kvcoord/batch_test.go b/pkg/kv/kvclient/kvcoord/batch_test.go index b0ccd8f83651..2fae3611dcaf 100644 --- a/pkg/kv/kvclient/kvcoord/batch_test.go +++ b/pkg/kv/kvclient/kvcoord/batch_test.go @@ -193,7 +193,7 @@ func TestBatchPrevNext(t *testing.T) { args.Key, args.EndKey = span.Key, span.EndKey ba.Add(args) } - if next, err := next(ba.Requests, roachpb.RKey(test.key)); err != nil { + if next, err := Next(ba.Requests, roachpb.RKey(test.key)); err != nil { t.Error(err) } else if !bytes.Equal(next, roachpb.Key(test.expFW)) { t.Errorf("next: expected %q, got %q", test.expFW, next) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 184cc6e1f55a..2fa16df88a8d 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -1318,7 +1318,7 @@ func (ds *DistSender) divideAndSendBatchToRanges( // one, and unless both descriptors are stale, the next descriptor's // StartKey would move us to the beginning of the current range, // resulting in a duplicate scan. - seekKey, err = next(ba.Requests, ri.Desc().EndKey) + seekKey, err = Next(ba.Requests, ri.Desc().EndKey) nextRS.Key = seekKey } if err != nil { @@ -1509,7 +1509,7 @@ func (ds *DistSender) sendPartialBatch( if err != nil { return response{pErr: roachpb.NewError(err)} } - ba.Requests, positions, err = truncate(ba.Requests, rs) + ba.Requests, positions, err = Truncate(ba.Requests, rs) if len(positions) == 0 && err == nil { // This shouldn't happen in the wild, but some tests exercise it. return response{ diff --git a/pkg/kv/kvclient/kvcoord/truncate_test.go b/pkg/kv/kvclient/kvcoord/truncate_test.go index 4bab7cdd144f..04127f59c25d 100644 --- a/pkg/kv/kvclient/kvcoord/truncate_test.go +++ b/pkg/kv/kvclient/kvcoord/truncate_test.go @@ -164,7 +164,7 @@ func TestTruncate(t *testing.T) { t.Errorf("%d: intersection failure: %v", i, err) continue } - reqs, pos, err := truncate(original.Requests, rs) + reqs, pos, err := Truncate(original.Requests, rs) if err != nil || test.err != "" { if !testutils.IsError(err, test.err) { t.Errorf("%d: %v (expected: %q)", i, err, test.err) diff --git a/pkg/kv/kvclient/kvstreamer/BUILD.bazel b/pkg/kv/kvclient/kvstreamer/BUILD.bazel new file mode 100644 index 000000000000..6718c4893851 --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/BUILD.bazel @@ -0,0 +1,56 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "kvstreamer", + srcs = [ + "avg_response_estimator.go", + "budget.go", + "streamer.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer", + visibility = ["//visibility:public"], + deps = [ + "//pkg/keys", + "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb:with-mocks", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/util/admission", + "//pkg/util/mon", + "//pkg/util/quotapool", + "//pkg/util/stop", + "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "kvstreamer_test", + srcs = [ + "avg_response_estimator_test.go", + "main_test.go", + "streamer_test.go", + ], + embed = [":kvstreamer"], + deps = [ + "//pkg/base", + "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", + "//pkg/kv/kvserver/concurrency/lock", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/skip", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/mon", + "//pkg/util/randutil", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go b/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go new file mode 100644 index 000000000000..56c45eff01d4 --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/avg_response_estimator.go @@ -0,0 +1,42 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer + +// avgResponseEstimator is a helper that estimates the average size of responses +// received by the Streamer. It is **not** thread-safe. +type avgResponseEstimator struct { + // responseBytes tracks the total footprint of all responses that the + // Streamer has already received. + responseBytes int64 + numResponses int64 +} + +// TODO(yuzefovich): use the optimizer-driven estimates. +const initialAvgResponseSize = 1 << 10 // 1KiB + +func (e *avgResponseEstimator) getAvgResponseSize() int64 { + if e.numResponses == 0 { + return initialAvgResponseSize + } + // TODO(yuzefovich): we currently use a simple average over the received + // responses, but it is likely to be suboptimal because it would be unfair + // to "large" batches that come in late (i.e. it would not be reactive + // enough). Consider using another function here. + return e.responseBytes / e.numResponses +} + +// update updates the actual information of the estimator based on numResponses +// responses that took up responseBytes bytes and correspond to a single +// BatchResponse. +func (e *avgResponseEstimator) update(responseBytes int64, numResponses int64) { + e.responseBytes += responseBytes + e.numResponses += numResponses +} diff --git a/pkg/kv/kvclient/kvstreamer/avg_response_estimator_test.go b/pkg/kv/kvclient/kvstreamer/avg_response_estimator_test.go new file mode 100644 index 000000000000..7c3337f59f26 --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/avg_response_estimator_test.go @@ -0,0 +1,56 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer + +import ( + "math" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestAvgResponseEstimator(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var e avgResponseEstimator + + // Before receiving any responses, we should be using the initial estimate. + require.Equal(t, int64(initialAvgResponseSize), e.getAvgResponseSize()) + + // Simulate receiving a single response. + firstResponseSize := int64(42) + e.update(firstResponseSize, 1) + // The estimate should now be exactly the size of that single response. + require.Equal(t, firstResponseSize, e.getAvgResponseSize()) + + // Simulate receiving 100 small BatchResponses. + smallResponseSize := int64(63) + for i := 0; i < 100; i++ { + e.update(smallResponseSize*5, 5) + } + // The estimate should now be pretty close to the size of a single response + // in the small BatchResponse. + diff := smallResponseSize - e.getAvgResponseSize() + require.True(t, math.Abs(float64(diff))/float64(smallResponseSize) < 0.05) + + // Now simulate receiving 10 large BatchResponses. + largeResponseSize := int64(17) + for i := 0; i < 10; i++ { + e.update(largeResponseSize*1000, 1000) + } + // The estimate should now be pretty close to the size of a single response + // in the large BatchResponse. + diff = largeResponseSize - e.getAvgResponseSize() + require.True(t, math.Abs(float64(diff))/float64(smallResponseSize) < 0.15) +} diff --git a/pkg/kv/kvclient/kvstreamer/budget.go b/pkg/kv/kvclient/kvstreamer/budget.go new file mode 100644 index 000000000000..44204c80af4c --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/budget.go @@ -0,0 +1,124 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// budget abstracts the memory budget that is provided to the Streamer by its +// client. +// +// This struct is a wrapper on top of mon.BoundAccount because we want to +// support the notion of budget "going in debt". This can occur in a degenerate +// case when a single large row exceeds the provided limit. The Streamer is +// expected to have only a single request in flight in this case. Additionally, +// the budget provides blocking (via waitCh) until it gets out of debt. +type budget struct { + mu struct { + syncutil.Mutex + // acc represents the current reservation of this budget against the + // root memory pool. + acc *mon.BoundAccount + } + // limitBytes is the maximum amount of bytes that this budget should reserve + // against acc, i.e. acc.Used() should not exceed limitBytes. However, in a + // degenerate case of a single large row, the budget can go into debt and + // acc.Used() might exceed limitBytes. + limitBytes int64 + // waitCh is used by the main loop of the workerCoordinator to block until + // available() becomes positive (until some release calls occur). + waitCh chan struct{} +} + +// newBudget creates a new budget with the specified limit. The limit determines +// the maximum amount of memory this budget is allowed to use (i.e. it'll be +// used lazily, as needed). +// +// The budget itself is responsible for staying under the limit, so acc should +// be bound to an unlimited memory monitor. This is needed in order to support +// the case of budget going into debt. Note that although it is an "unlimited +// memory monitor", the monitor is still limited by --max-sql-memory in size +// eventually because all monitors are descendants of the root SQL monitor. +// +// The budget takes ownership of the memory account, and the caller is allowed +// to interact with the account only after canceling the Streamer (because +// memory accounts are not thread-safe). +func newBudget(acc *mon.BoundAccount, limitBytes int64) *budget { + b := budget{ + limitBytes: limitBytes, + waitCh: make(chan struct{}), + } + b.mu.acc = acc + return &b +} + +// available returns how many bytes are currently available in the budget. The +// answer can be negative, in case the Streamer has used un-budgeted memory +// (e.g. one result was very large putting the budget in debt). +// +// Note that it's possible that actually available budget is less than the +// number returned - this might occur if --max-sql-memory root pool is almost +// used up. +func (b *budget) available() int64 { + b.mu.Lock() + defer b.mu.Unlock() + return b.limitBytes - b.mu.acc.Used() +} + +// consume draws bytes from the available budget. An error is returned if the +// root pool budget is used up such that the budget's limit cannot be fully +// reserved. +// - allowDebt indicates whether the budget is allowed to go into debt on this +// consumption. In other words, if allowDebt is true, then acc's reservation is +// allowed to exceed limitBytes (but the error is still returned if the root +// pool budget is exceeded). Note that allowDebt value applies only to this +// consume() call and is not carried forward. +// +// b's mutex should not be held when calling this method. +func (b *budget) consume(ctx context.Context, bytes int64, allowDebt bool) error { + b.mu.Lock() + defer b.mu.Unlock() + return b.consumeLocked(ctx, bytes, allowDebt) +} + +// consumeLocked is the same as consume but assumes that the b's lock is held. +func (b *budget) consumeLocked(ctx context.Context, bytes int64, allowDebt bool) error { + b.mu.AssertHeld() + // If we're asked to not exceed the limit (and the limit is greater than + // five bytes - limits of five bytes or less are treated as a special case + // for "forced disk spilling" scenarios like in logic tests), we have to + // check whether we'll stay within the budget. + if !allowDebt && b.limitBytes > 5 { + if b.mu.acc.Used()+bytes > b.limitBytes { + return mon.MemoryResource.NewBudgetExceededError(bytes, b.mu.acc.Used(), b.limitBytes) + } + } + return b.mu.acc.Grow(ctx, bytes) +} + +// release returns bytes to the available budget. +func (b *budget) release(ctx context.Context, bytes int64) { + b.mu.Lock() + defer b.mu.Unlock() + b.mu.acc.Shrink(ctx, bytes) + if b.limitBytes > b.mu.acc.Used() { + // Since we now have some available budget, we non-blockingly send on + // the wait channel to notify the mainCoordinator about it. + select { + case b.waitCh <- struct{}{}: + default: + } + } +} diff --git a/pkg/kv/kvclient/kvstreamer/main_test.go b/pkg/kv/kvclient/kvstreamer/main_test.go new file mode 100644 index 000000000000..40dc560be5f8 --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + os.Exit(m.Run()) +} diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go new file mode 100644 index 000000000000..dd7a82bd1d1b --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/streamer.go @@ -0,0 +1,1165 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer + +import ( + "context" + "runtime" + "sort" + "sync" + "unsafe" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/admission" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/quotapool" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" +) + +// OperationMode describes the mode of operation of the Streamer. +type OperationMode int + +const ( + _ OperationMode = iota + // InOrder is the mode of operation in which the results are delivered in + // the order in which the requests were handed off to the Streamer. This + // mode forces the Streamer to buffer the results it produces through its + // internal parallel execution of the requests. Since the results of the + // concurrent requests can come in an arbitrary order, they are buffered and + // might end up being dropped (resulting in wasted/duplicate work) to make + // space for the results at the front of the line. This would occur when the + // budget limitBytes is reached and the size estimates that lead to too much + // concurrency in the execution were wrong. + InOrder + // OutOfOrder is the mode of operation in which the results are delivered in + // the order in which they're produced. The caller will use the keys field + // of each Result to associate it with the corresponding requests. This mode + // of operation lets the Streamer reuse the memory budget as quickly as + // possible. + OutOfOrder +) + +// Remove an unused warning for now. +// TODO(yuzefovich): remove this when supported. +var _ = InOrder + +// Result describes the result of performing a single KV request. +type Result struct { + // GetResp and ScanResp represent the response to a request. Only one of the + // two will be populated. + // + // The responses are to be considered immutable; the Streamer might hold on + // to the respective memory. Calling MemoryTok.Release() tells the Streamer + // that the response is no longer needed. + GetResp *roachpb.GetResponse + // ScanResp can contain a partial response to a ScanRequest (when Complete + // is false). In that case, there will be a further result with the + // continuation; that result will use the same Key. Notably, SQL rows will + // never be split across multiple results. + ScanResp struct { + *roachpb.ScanResponse + // If the Result represents a scan result, Complete indicates whether + // this is the last response for the respective scan, or if there are + // more responses to come. In any case, ScanResp never contains partial + // rows (i.e. a single row is never split into different Results). + // + // When running in InOrder mode, Results for a single scan will be + // delivered in key order (in addition to results for different scans + // being delivered in request order). When running in OutOfOrder mode, + // Results for a single scan can be delivered out of key order (in + // addition to results for different scans being delivered out of + // request order). + Complete bool + } + // EnqueueKeysSatisfied identifies the requests that this Result satisfies. + // In OutOfOrder mode, a single Result can satisfy multiple identical + // requests. In InOrder mode a Result can only satisfy multiple consecutive + // requests. + EnqueueKeysSatisfied []int + // MemoryTok.Release() needs to be called by the recipient once it's not + // referencing this Result any more. If this was the last (or only) + // reference to this Result, the memory used by this Result is made + // available in the Streamer's budget. + // + // Internally, Results are refcounted. Multiple Results referencing the same + // GetResp/ScanResp can be returned from separate `GetResults()` calls, and + // the Streamer internally does buffering and caching of Results - which + // also contributes to the refcounts. + MemoryTok ResultMemoryToken + // position tracks the ordinal among all originally enqueued requests that + // this result satisfies. See singleRangeBatch.positions for more details. + // + // If Streamer.Enqueue() was called with nil enqueueKeys argument, then + // EnqueueKeysSatisfied will exactly contain position; if non-nil + // enqueueKeys argument was passed, then position is used as an ordinal to + // lookup into enqueueKeys to populate EnqueueKeysSatisfied. + // TODO(yuzefovich): this might need to be []int when non-unique requests + // are supported. + position int +} + +// ResultMemoryToken represents a handle to a Result's memory tracking. The +// recipient of a Result is required to call Release() when the Result is not in +// use any more so that its memory is returned to the Streamer's budget. +type ResultMemoryToken interface { + // Release decrements the refcount. + Release(context.Context) +} + +// Hints provides different hints to the Streamer for optimization purposes. +type Hints struct { + // UniqueRequests tells the Streamer that the requests will be unique. As + // such, there's no point in de-duping them or caching results. + UniqueRequests bool +} + +type resultMemoryToken struct { + budget *budget + toRelease int64 +} + +var _ ResultMemoryToken = &resultMemoryToken{} + +func (t *resultMemoryToken) Release(ctx context.Context) { + t.budget.release(ctx, t.toRelease) +} + +// Streamer provides a streaming oriented API for reading from the KV layer. At +// the moment the Streamer only works when SQL rows are comprised of a single KV +// (i.e. a single column family). +// TODO(yuzefovich): lift the restriction on a single column family once KV is +// updated so that rows are never split across different BatchResponses when +// TargetBytes limitBytes is exceeded. +// +// The example usage is roughly as follows: +// +// s := NewStreamer(...) +// s.Init(OperationMode, Hints) +// ... +// for needMoreKVs { +// // Check whether there are results to the previously enqueued requests. +// // This will block if no results are available, but there are some +// // enqueued requests. +// results, err := s.GetResults(ctx) +// // err check +// ... +// if len(results) > 0 { +// processResults(results) +// // return to the client +// } +// // All previously enqueued requests have already been responded to. +// if moreRequestsToEnqueue { +// err := s.Enqueue(ctx, requests, enqueueKeys) +// // err check +// ... +// } else { +// // done +// ... +// } +// } +// ... +// s.Close() +// +// The Streamer builds on top of the BatchRequest API provided by the DistSender +// and aims to allow for executing the requests in parallel (to improve the +// performance) while setting the memory limits on those requests (for stability +// purposes). +// +// The parallelism is achieved by splitting the incoming requests into +// single-range batches where each such batch will hit a fast-path in the +// DistSender (unless there have been changes to range boundaries). Since these +// batches are executed concurrently, the LeafTxns are used. +// +// The memory limit handling is achieved by the Streamer guessing the size of +// the response for each request and setting TargetBytes accordingly. The +// concurrency of the Streamer is limited by its memory limit. +// +// The Streamer additionally utilizes different optimizations to improve the +// performance: +// - when possible, sorting requests in key order to take advantage of low-level +// Pebble locality optimizations +// - when necessary, buffering the responses received out of order +// - when necessary, caching the responses to short-circuit repeated lookups. +// TODO(yuzefovich): add an optimization of transparent refreshes when there is +// a single Streamer in the local flow. +// TODO(yuzefovich): support pipelining of Enqueue and GetResults calls. +type Streamer struct { + distSender *kvcoord.DistSender + stopper *stop.Stopper + + mode OperationMode + hints Hints + budget *budget + + coordinator workerCoordinator + coordinatorStarted bool + coordinatorCtxCancel context.CancelFunc + + waitGroup sync.WaitGroup + + enqueueKeys []int + + // waitForResults is used to block GetResults() call until some results are + // available. + waitForResults chan struct{} + + mu struct { + syncutil.Mutex + + avgResponseEstimator avgResponseEstimator + + // requestsToServe contains all single-range sub-requests that have yet + // to be served. + // TODO(yuzefovich): consider using ring.Buffer instead of a slice. + requestsToServe []singleRangeBatch + + // numRangesLeftPerScanRequest tracks how many ranges a particular + // originally enqueued ScanRequest touches, but scanning of those ranges + // isn't complete. It is allocated lazily when the first ScanRequest is + // encountered in Enqueue. + numRangesLeftPerScanRequest []int + + // numEnqueuedRequests tracks the number of the originally enqueued + // requests. + numEnqueuedRequests int + + // numCompleteRequests tracks the number of the originally enqueued + // requests that have already been completed. + numCompleteRequests int + + // numRequestsInFlight tracks the number of single-range batches that + // are currently being served asynchronously (i.e. those that have + // already left requestsToServe queue, but for which we haven't received + // the results yet). + // TODO(yuzefovich): check whether the contention on mu when accessing + // this field is sufficient to justify pulling it out into an atomic. + numRequestsInFlight int + + // results are the results of already completed requests that haven't + // been returned by GetResults() yet. + results []Result + err error + } +} + +// streamerConcurrencyLimit is an upper bound on the number of asynchronous +// requests that a single Streamer can have in flight. The default value for +// this setting is chosen arbitrarily as 1/8th of the default value for the +// senderConcurrencyLimit. +var streamerConcurrencyLimit = settings.RegisterIntSetting( + settings.TenantWritable, + "kv.streamer.concurrency_limit", + "maximum number of asynchronous requests by a single streamer", + max(128, int64(8*runtime.GOMAXPROCS(0))), + settings.NonNegativeInt, +) + +func max(a, b int64) int64 { + if a > b { + return a + } + return b +} + +// NewStreamer creates a new Streamer. +// +// limitBytes determines the maximum amount of memory this Streamer is allowed +// to use (i.e. it'll be used lazily, as needed). The more memory it has, the +// higher its internal concurrency and throughput. +// +// acc should be bound to an unlimited memory monitor, and the Streamer itself +// is responsible for staying under the limitBytes. +// +// The Streamer takes ownership of the memory account, and the caller is allowed +// to interact with the account only after canceling the Streamer (because +// memory accounts are not thread-safe). +func NewStreamer( + distSender *kvcoord.DistSender, + stopper *stop.Stopper, + txn *kv.Txn, + st *cluster.Settings, + lockWaitPolicy lock.WaitPolicy, + limitBytes int64, + acc *mon.BoundAccount, +) *Streamer { + s := &Streamer{ + distSender: distSender, + stopper: stopper, + budget: newBudget(acc, limitBytes), + } + s.coordinator = workerCoordinator{ + s: s, + txn: txn, + lockWaitPolicy: lockWaitPolicy, + requestAdmissionHeader: txn.AdmissionHeader(), + responseAdmissionQ: txn.DB().SQLKVResponseAdmissionQ, + } + // TODO(yuzefovich): consider lazily allocating this IntPool only when + // enqueued requests span multiple batches. + s.coordinator.asyncSem = quotapool.NewIntPool( + "single Streamer async concurrency", + uint64(streamerConcurrencyLimit.Get(&st.SV)), + ) + s.coordinator.mu.hasWork = sync.NewCond(&s.coordinator.mu) + streamerConcurrencyLimit.SetOnChange(&st.SV, func(ctx context.Context) { + s.coordinator.asyncSem.UpdateCapacity(uint64(streamerConcurrencyLimit.Get(&st.SV))) + }) + stopper.AddCloser(s.coordinator.asyncSem.Closer("stopper")) + return s +} + +// Init initializes the Streamer. +// +// OperationMode controls the order in which results are delivered to the +// client. When possible, prefer OutOfOrder mode. +// +// Hints can be used to hint the aggressiveness of the caching policy. In +// particular, it can be used to disable caching when the client knows that all +// looked-up keys are unique (e.g. in the case of an index-join). +func (s *Streamer) Init(mode OperationMode, hints Hints) { + if mode != OutOfOrder { + panic(errors.AssertionFailedf("only OutOfOrder mode is supported")) + } + s.mode = mode + if !hints.UniqueRequests { + panic(errors.AssertionFailedf("only unique requests are currently supported")) + } + s.hints = hints + s.waitForResults = make(chan struct{}, 1) +} + +// Enqueue dispatches multiple requests for execution. Results are delivered +// through the GetResults call. If enqueueKeys is not nil, it needs to contain +// one ID for each request; responses will reference that ID so that the client +// can associate them to the requests. If enqueueKeys is nil, then the responses +// will reference the ordinals of the corresponding requests among reqs. +// +// Multiple requests can specify the same key. In this case, their respective +// responses will also reference the same key. This is useful, for example, for +// "range-based lookup joins" where multiple spans are read in the context of +// the same input-side row (see multiSpanGenerator implementation of +// rowexec.joinReaderSpanGenerator interface for more details). +// +// The Streamer takes over the given requests, will perform the memory +// accounting against its budget and might modify the requests in place. +// +// In InOrder operation mode, responses will be delivered in reqs order. +// +// It is the caller's responsibility to ensure that the memory footprint of reqs +// (i.e. roachpb.Spans inside of the requests) is reasonable. Enqueue will +// return an error if that footprint exceeds the Streamer's limitBytes. The +// exception is made only when a single request is enqueued in order to allow +// the caller to proceed when the key to lookup is arbitrarily large. As a rule +// of thumb though, the footprint of reqs should be on the order of MBs, and not +// tens of MBs. +// +// Currently, enqueuing new requests while there are still requests in progress +// from the previous invocation is prohibited. +// TODO(yuzefovich): lift this restriction and introduce the pipelining. +func (s *Streamer) Enqueue( + ctx context.Context, reqs []roachpb.RequestUnion, enqueueKeys []int, +) (retErr error) { + if !s.coordinatorStarted { + var coordinatorCtx context.Context + coordinatorCtx, s.coordinatorCtxCancel = context.WithCancel(ctx) + s.waitGroup.Add(1) + if err := s.stopper.RunAsyncTask(coordinatorCtx, "streamer-coordinator", s.coordinator.mainLoop); err != nil { + // The new goroutine wasn't spun up, so mainLoop won't get executed + // and we have to decrement the wait group ourselves. + s.waitGroup.Done() + return err + } + s.coordinatorStarted = true + } + + // TODO(yuzefovich): we might want to have more fine-grained lock + // acquisitions once pipelining is implemented. + s.mu.Lock() + defer func() { + if retErr != nil && s.mu.err == nil { + // Set the error so that mainLoop of the worker coordinator exits + // as soon as possible, without issuing any more requests. + s.mu.err = retErr + } + s.mu.Unlock() + }() + + if enqueueKeys != nil && len(enqueueKeys) != len(reqs) { + return errors.AssertionFailedf("invalid enqueueKeys: len(reqs) = %d, len(enqueueKeys) = %d", len(reqs), len(enqueueKeys)) + } + s.enqueueKeys = enqueueKeys + + if s.mu.numEnqueuedRequests != s.mu.numCompleteRequests { + return errors.AssertionFailedf("Enqueue is called before the previous requests have been completed") + } + if len(s.mu.results) > 0 { + return errors.AssertionFailedf("Enqueue is called before the results of the previous requests have been retrieved") + } + + s.mu.numEnqueuedRequests = len(reqs) + s.mu.numCompleteRequests = 0 + + // The minimal key range encompassing all requests contained within. + // Local addressing has already been resolved. + rs, err := keys.Range(reqs) + if err != nil { + return err + } + + // Divide the given requests into single-range batches that are added to + // requestsToServe, and the worker coordinator will then pick those batches + // up to execute asynchronously. + var totalReqsMemUsage int64 + // TODO(yuzefovich): in InOrder mode we need to treat the head-of-the-line + // request differently. + seekKey := rs.Key + const scanDir = kvcoord.Ascending + ri := kvcoord.MakeRangeIterator(s.distSender) + ri.Seek(ctx, seekKey, scanDir) + if !ri.Valid() { + return ri.Error() + } + firstScanRequest := true + for ; ri.Valid(); ri.Seek(ctx, seekKey, scanDir) { + // Truncate the request span to the current range. + singleRangeSpan, err := rs.Intersect(ri.Token().Desc()) + if err != nil { + return err + } + // Find all requests that touch the current range. + singleRangeReqs, positions, err := kvcoord.Truncate(reqs, singleRangeSpan) + if err != nil { + return err + } + for _, pos := range positions { + if _, isScan := reqs[pos].GetInner().(*roachpb.ScanRequest); isScan { + if firstScanRequest { + // We have some ScanRequests, so we have to set up + // numRangesLeftPerScanRequest. + if cap(s.mu.numRangesLeftPerScanRequest) < len(reqs) { + s.mu.numRangesLeftPerScanRequest = make([]int, len(reqs)) + } else { + // We can reuse numRangesLeftPerScanRequest allocated on + // the previous call to Enqueue after we zero it out. + s.mu.numRangesLeftPerScanRequest = s.mu.numRangesLeftPerScanRequest[:len(reqs)] + for n := 0; n < len(s.mu.numRangesLeftPerScanRequest); { + n += copy(s.mu.numRangesLeftPerScanRequest[n:], zeroIntSlice) + } + } + } + s.mu.numRangesLeftPerScanRequest[pos]++ + firstScanRequest = false + } + } + + // TODO(yuzefovich): perform the de-duplication here. + //if !s.hints.UniqueRequests { + //} + + r := singleRangeBatch{ + reqs: singleRangeReqs, + positions: positions, + reqsReservedBytes: requestsMemUsage(singleRangeReqs), + } + totalReqsMemUsage += r.reqsReservedBytes + + if s.mode == OutOfOrder { + // Sort all single-range requests to be in the key order. + sort.Sort(&r) + } + + s.mu.requestsToServe = append(s.mu.requestsToServe, r) + + // Determine next seek key, taking potentially sparse requests into + // consideration. + // + // In next iteration, query next range. + // It's important that we use the EndKey of the current descriptor + // as opposed to the StartKey of the next one: if the former is stale, + // it's possible that the next range has since merged the subsequent + // one, and unless both descriptors are stale, the next descriptor's + // StartKey would move us to the beginning of the current range, + // resulting in a duplicate scan. + seekKey, err = kvcoord.Next(reqs, ri.Desc().EndKey) + rs.Key = seekKey + if err != nil { + return err + } + } + + // Account for the memory used by all the requests. We allow the budget to + // go into debt iff a single request was enqueued. This is needed to support + // the case of arbitrarily large keys - the caller is expected to produce + // requests with such cases one at a time. + allowDebt := len(reqs) == 1 + if err = s.budget.consume(ctx, totalReqsMemUsage, allowDebt); err != nil { + return err + } + + // TODO(yuzefovich): it might be better to notify the coordinator once + // one singleRangeBatch object has been appended to s.mu.requestsToServe. + s.coordinator.mu.hasWork.Signal() + return nil +} + +// GetResults blocks until at least one result is available. If the operation +// mode is OutOfOrder, any result will do, and the caller is expected to examine +// Result.EnqueueKeysSatisfied to understand which request the result +// corresponds to. For InOrder, only head-of-line results will do. Zero-length +// result slice is returned once all enqueued requests have been responded to. +func (s *Streamer) GetResults(ctx context.Context) ([]Result, error) { + s.mu.Lock() + results := s.mu.results + err := s.mu.err + s.mu.results = nil + allComplete := s.mu.numCompleteRequests == s.mu.numEnqueuedRequests + // Non-blockingly clear the waitForResults channel in case we've just picked + // up some results. We do so while holding the mutex so that new results + // aren't appended. + select { + case <-s.waitForResults: + default: + } + s.mu.Unlock() + + if len(results) > 0 || allComplete || err != nil { + return results, err + } + + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-s.waitForResults: + s.mu.Lock() + results = s.mu.results + err = s.mu.err + s.mu.results = nil + s.mu.Unlock() + return results, err + } +} + +// notifyGetResultsLocked non-blockingly sends a message on waitForResults +// channel. This method should be called only while holding the lock of s.mu so +// that other results couldn't be appended which would cause us to miss the +// notification about that. +func (s *Streamer) notifyGetResultsLocked() { + s.mu.AssertHeld() + select { + case s.waitForResults <- struct{}{}: + default: + } +} + +// setError sets the error on the Streamer if no error has been set previously +// and unblocks GetResults() if needed. +// +// The mutex of s must not be already held. +func (s *Streamer) setError(err error) { + s.mu.Lock() + defer s.mu.Unlock() + if s.mu.err == nil { + s.mu.err = err + } + s.notifyGetResultsLocked() +} + +// Close cancels all in-flight operations and releases all of the resources of +// the Streamer. It blocks until all goroutines created by the Streamer exit. No +// other calls on s are allowed after this. +func (s *Streamer) Close() { + if s.coordinatorStarted { + s.coordinatorCtxCancel() + s.coordinator.mu.Lock() + s.coordinator.mu.done = true + // Unblock the coordinator in case it is waiting for more work. + s.coordinator.mu.hasWork.Signal() + s.coordinator.mu.Unlock() + } + s.waitGroup.Wait() + *s = Streamer{} +} + +// getNumRequestsInFlight returns the number of requests that are currently in +// flight. This method should be called without holding the lock of s. +func (s *Streamer) getNumRequestsInFlight() int { + s.mu.Lock() + defer s.mu.Unlock() + return s.mu.numRequestsInFlight +} + +// adjustNumRequestsInFlight updates the number of requests that are currently +// in flight. This method should be called without holding the lock of s. +func (s *Streamer) adjustNumRequestsInFlight(delta int) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.numRequestsInFlight += delta +} + +// singleRangeBatch contains parts of the originally enqueued requests that have +// been truncated to be within a single range. All requests within the +// singleRangeBatch will be issued as a single BatchRequest. +type singleRangeBatch struct { + reqs []roachpb.RequestUnion + // positions is a 1-to-1 mapping with reqs to indicate which ordinal among + // the originally enqueued requests a particular reqs[i] corresponds to. In + // other words, if reqs[i] is (or a part of) enqueuedReqs[j], then + // positions[i] = j. + // TODO(yuzefovich): this might need to be [][]int when non-unique requests + // are supported. + positions []int + // reqsReservedBytes tracks the memory reservation against the budget for + // the memory usage of reqs. + reqsReservedBytes int64 +} + +var _ sort.Interface = &singleRangeBatch{} + +func (r *singleRangeBatch) Len() int { + return len(r.reqs) +} + +func (r *singleRangeBatch) Swap(i, j int) { + r.reqs[i], r.reqs[j] = r.reqs[j], r.reqs[i] + r.positions[i], r.positions[j] = r.positions[j], r.positions[i] +} + +// Less returns true if r.reqs[i]'s key comes before r.reqs[j]'s key. +func (r *singleRangeBatch) Less(i, j int) bool { + // TODO(yuzefovich): figure out whether it's worth extracting the keys when + // constructing singleRangeBatch object. + return r.reqs[i].GetInner().Header().Key.Compare(r.reqs[j].GetInner().Header().Key) < 0 +} + +type workerCoordinator struct { + s *Streamer + txn *kv.Txn + lockWaitPolicy lock.WaitPolicy + + asyncSem *quotapool.IntPool + + // For request and response admission control. + requestAdmissionHeader roachpb.AdmissionHeader + responseAdmissionQ *admission.WorkQueue + + mu struct { + syncutil.Mutex + hasWork *sync.Cond + // done is set to true once the Streamer is closed meaning the worker + // coordinator must exit. + done bool + } +} + +// mainLoop runs throughout the lifetime of the Streamer (from the first Enqueue +// call until Cancel) and routes the single-range batches for asynchronous +// execution. This function is dividing up the Streamer's budget for each of +// those batches and won't start executing the batches if the available budget +// is insufficient. The function exits when an error is encountered by one of +// the asynchronous requests. +func (w *workerCoordinator) mainLoop(ctx context.Context) { + defer w.s.waitGroup.Done() + for { + // Get next requests to serve. + requestsToServe, avgResponseSize, shouldExit := w.getRequests() + if shouldExit { + return + } + if len(requestsToServe) == 0 { + // If the Streamer isn't closed yet, block until there are enqueued + // requests. + w.mu.Lock() + if !w.mu.done { + w.mu.hasWork.Wait() + } + w.mu.Unlock() + if ctx.Err() != nil { + w.s.setError(ctx.Err()) + return + } + continue + } + + // Now wait until there is enough budget to at least receive one full + // response (but only if there are requests in flight - if there are + // none, then we might have a degenerate case when a single row is + // expected to exceed the budget). + // TODO(yuzefovich): consider using a multiple of avgResponseSize here. + for w.s.getNumRequestsInFlight() > 0 && w.s.budget.available() < avgResponseSize { + select { + case <-w.s.budget.waitCh: + case <-ctx.Done(): + w.s.setError(ctx.Err()) + return + } + } + + err := w.issueRequestsForAsyncProcessing(ctx, requestsToServe, avgResponseSize) + if err != nil { + w.s.setError(err) + return + } + } +} + +// getRequests returns all currently enqueued requests to be served. +// +// A boolean that indicates whether the coordinator should exit is returned. +func (w *workerCoordinator) getRequests() ( + requestsToServe []singleRangeBatch, + avgResponseSize int64, + shouldExit bool, +) { + w.s.mu.Lock() + defer w.s.mu.Unlock() + requestsToServe = w.s.mu.requestsToServe + avgResponseSize = w.s.mu.avgResponseEstimator.getAvgResponseSize() + shouldExit = w.s.mu.err != nil + return requestsToServe, avgResponseSize, shouldExit +} + +// issueRequestsForAsyncProcessing iterates over the given requests and issues +// them to be served asynchronously while there is enough budget available to +// receive the responses. Once the budget is exhausted, no new requests are +// issued, the only exception is made for the case when there are no other +// requests in flight, and in that scenario, a single request will be issued. +// +// It is assumed that requestsToServe is a prefix of w.s.mu.requestsToServe +// (i.e. it is possible that some other requests have been appended to +// w.s.mu.requestsToServe after requestsToServe have been grabbed). All issued +// requests are removed from w.s.mu.requestToServe. +func (w *workerCoordinator) issueRequestsForAsyncProcessing( + ctx context.Context, requestsToServe []singleRangeBatch, avgResponseSize int64, +) error { + var numRequestsIssued int + defer func() { + w.s.mu.Lock() + // We can just slice here since we only append to requestToServe at + // the moment. + w.s.mu.requestsToServe = w.s.mu.requestsToServe[numRequestsIssued:] + w.s.mu.Unlock() + }() + w.s.budget.mu.Lock() + defer w.s.budget.mu.Unlock() + + headOfLine := w.s.getNumRequestsInFlight() == 0 + var budgetIsExhausted bool + for numRequestsIssued < len(requestsToServe) && !budgetIsExhausted { + availableBudget := w.s.budget.limitBytes - w.s.budget.mu.acc.Used() + if availableBudget < avgResponseSize { + if !headOfLine { + // We don't have enough budget available to serve this request, + // and there are other requests in flight, so we'll wait for + // some of them to finish. + break + } + budgetIsExhausted = true + if availableBudget < 1 { + // The budget is already in debt, and we have no requests in + // flight. This occurs when we have very large roachpb.Span in + // the request. In such a case, we still want to make progress + // by giving the smallest TargetBytes possible while asking the + // KV layer to not return an empty response. + availableBudget = 1 + } + } + singleRangeReqs := requestsToServe[numRequestsIssued] + // Calculate what TargetBytes limit to use for the BatchRequest that + // will be issued based on singleRangeReqs. We use the estimate to guess + // how much memory the response will need, and we reserve this + // estimation up front. + // + // Note that TargetBytes will be a strict limit on the response size + // (except in a degenerate case for head-of-the-line request that will + // get a very large single row in response which will exceed this + // limit). + targetBytes := int64(len(singleRangeReqs.reqs)) * avgResponseSize + if targetBytes > availableBudget { + // The estimate tells us that we don't have enough budget to receive + // the full response; however, in order to utilize the available + // budget fully, we can still issue this request with the truncated + // TargetBytes value hoping to receive a partial response. + targetBytes = availableBudget + } + if err := w.s.budget.consumeLocked(ctx, targetBytes, headOfLine /* allowDebt */); err != nil { + // This error cannot be because of the budget going into debt. If + // headOfLine is true, then we're allowing debt; otherwise, we have + // truncated targetBytes above to not exceed availableBudget, and + // we're holding the budget's mutex. Thus, the error indicates that + // the root memory pool has been exhausted. + if !headOfLine { + // There are some requests in flight, so we'll let them finish. + // + // This is opportunistic behavior where we're hoping that once + // other requests are fully processed (i.e. the corresponding + // results are Release()'d), we'll be able to make progress on + // this request too, without exceeding the root memory pool. + // + // We're not really concerned about pushing the node towards the + // OOM situation because we're still staying within the root + // memory pool limit (which should have some safety gap with the + // available RAM). Furthermore, if other queries are consuming + // all of the root memory pool limit, then the head-of-the-line + // request will notice it and will exit accordingly. + break + } + // We don't have any requests in flight, so we'll exit to be safe + // (in order not to OOM the node). Most likely this occurs when + // there are concurrent memory-intensive queries which this Streamer + // has no control over. + // + // We could have issued this head-of-the-line request with lower + // targetBytes value (unless it is already 1), but the fact that the + // root memory pool is exhausted indicates that the node might be + // overloaded already, so it seems better to not ask it to receive + // any more responses at the moment. + return err + } + w.performRequestAsync(ctx, singleRangeReqs, targetBytes, headOfLine) + numRequestsIssued++ + headOfLine = false + } + return nil +} + +// addRequest adds a single-range batch to be processed later. +func (w *workerCoordinator) addRequest(req singleRangeBatch) { + w.s.mu.Lock() + defer w.s.mu.Unlock() + w.s.mu.requestsToServe = append(w.s.mu.requestsToServe, req) + w.mu.hasWork.Signal() +} + +func (w *workerCoordinator) asyncRequestCleanup() { + w.s.adjustNumRequestsInFlight(-1 /* delta */) + w.s.waitGroup.Done() +} + +// performRequestAsync dispatches the given single-range batch for evaluation +// asynchronously. If the batch cannot be evaluated fully (due to exhausting its +// memory limitBytes), the "resume" single-range batch will be added into +// requestsToServe, and mainLoop will pick that up to process later. +// +// targetBytes specifies the memory budget that this single-range batch should +// be issued with. targetBytes bytes have already been consumed from the budget, +// and this amount of memory is owned by the goroutine that is spun up to +// perform the request. Once the response is received, performRequestAsync +// reconciles the budget so that the actual footprint of the response is +// consumed. Each Result produced based on that response will track a part of +// the memory reservation (according to the Result's footprint) that will be +// returned back to the budget once Result.MemoryTok.Release is called. +// +// headOfLine indicates whether this request is the current head of the line. +// Head-of-the-line requests are treated specially in a sense that they are +// allowed to put the budget into debt. The caller is responsible for ensuring +// that there is at most one asynchronous request with headOfLine=true at all +// times. +func (w *workerCoordinator) performRequestAsync( + ctx context.Context, req singleRangeBatch, targetBytes int64, headOfLine bool, +) { + w.s.waitGroup.Add(1) + w.s.adjustNumRequestsInFlight(1 /* delta */) + if err := w.s.stopper.RunAsyncTaskEx( + ctx, + stop.TaskOpts{ + TaskName: "streamer-lookup-async", + Sem: w.asyncSem, + WaitForSem: true, + }, + func(ctx context.Context) { + defer w.asyncRequestCleanup() + var ba roachpb.BatchRequest + ba.Header.WaitPolicy = w.lockWaitPolicy + ba.Header.TargetBytes = targetBytes + ba.Header.TargetBytesAllowEmpty = !headOfLine + // TODO(yuzefovich): consider setting MaxSpanRequestKeys whenever + // applicable (#67885). + ba.AdmissionHeader = w.requestAdmissionHeader + // We always have some memory reserved against the memory account, + // regardless of the value of headOfLine. + ba.AdmissionHeader.NoMemoryReservedAtSource = false + ba.Requests = req.reqs + + // TODO(yuzefovich): in Enqueue we split all requests into + // single-range batches, so ideally ba touches a single range in + // which case we hit the fast path in the DistSender. However, if + // the range boundaries have changed after we performed the split + // (or we had stale range cache at the time of the split), the + // DistSender will transparently re-split ba into several + // sub-batches that will be executed sequentially because of the + // presence of limits. We could, instead, ask the DistSender to not + // perform that re-splitting and return an error, then we'll rely on + // the updated range cache to perform re-splitting ourselves. This + // should offer some performance improvements since we'd eliminate + // unnecessary blocking (due to sequential evaluation of sub-batches + // by the DistSender). For the initial implementation it doesn't + // seem important though. + br, err := w.txn.Send(ctx, ba) + if err != nil { + // TODO(yuzefovich): if err is + // ReadWithinUncertaintyIntervalError and there is only a single + // Streamer in a single local flow, attempt to transparently + // refresh. + w.s.setError(err.GoError()) + return + } + + var resumeReq singleRangeBatch + // We will reuse the slices for the resume spans, if any. + resumeReq.reqs = req.reqs[:0] + resumeReq.positions = req.positions[:0] + var results []Result + var numCompleteGetResponses int + // memoryFootprintBytes tracks the total memory footprint of + // non-empty responses. This will be equal to the sum of the all + // resultMemoryTokens created. + var memoryFootprintBytes int64 + var hasNonEmptyScanResponse bool + for i, resp := range br.Responses { + enqueueKey := req.positions[i] + if w.s.enqueueKeys != nil { + enqueueKey = w.s.enqueueKeys[req.positions[i]] + } + reply := resp.GetInner() + origReq := req.reqs[i] + // Unset the original request so that we lose the reference to + // the span. + req.reqs[i] = roachpb.RequestUnion{} + switch origRequest := origReq.GetInner().(type) { + case *roachpb.GetRequest: + get := reply.(*roachpb.GetResponse) + if get.ResumeSpan != nil { + // This Get wasn't completed - update the original + // request according to the ResumeSpan and include it + // into the batch again. + origRequest.SetSpan(*get.ResumeSpan) + resumeReq.reqs = append(resumeReq.reqs, origReq) + resumeReq.positions = append(resumeReq.positions, req.positions[i]) + } else { + // This Get was completed. + toRelease := int64(get.Size()) + result := Result{ + GetResp: get, + // This currently only works because all requests + // are unique. + EnqueueKeysSatisfied: []int{enqueueKey}, + MemoryTok: &resultMemoryToken{ + toRelease: toRelease, + budget: w.s.budget, + }, + position: req.positions[i], + } + memoryFootprintBytes += toRelease + results = append(results, result) + numCompleteGetResponses++ + } + + case *roachpb.ScanRequest: + scan := reply.(*roachpb.ScanResponse) + resumeSpan := scan.ResumeSpan + if len(scan.Rows) > 0 || len(scan.BatchResponses) > 0 { + toRelease := int64(scan.Size()) + result := Result{ + // This currently only works because all requests + // are unique. + EnqueueKeysSatisfied: []int{enqueueKey}, + MemoryTok: &resultMemoryToken{ + toRelease: toRelease, + budget: w.s.budget, + }, + position: req.positions[i], + } + result.ScanResp.ScanResponse = scan + // Complete field will be set below. + memoryFootprintBytes += toRelease + results = append(results, result) + hasNonEmptyScanResponse = true + } + if resumeSpan != nil { + // This Scan wasn't completed - update the original + // request according to the resumeSpan and include it + // into the batch again. + origRequest.SetSpan(*resumeSpan) + resumeReq.reqs = append(resumeReq.reqs, origReq) + resumeReq.positions = append(resumeReq.positions, req.positions[i]) + } + } + } + + // Now adjust the budget based on the actual memory footprint of + // non-empty responses as well as resume spans, if any. + respOverestimate := targetBytes - memoryFootprintBytes + var reqsMemUsage int64 + if len(resumeReq.reqs) > 0 { + reqsMemUsage = requestsMemUsage(resumeReq.reqs) + } + reqOveraccounted := req.reqsReservedBytes - reqsMemUsage + overaccountedTotal := respOverestimate + reqOveraccounted + if overaccountedTotal >= 0 { + w.s.budget.release(ctx, overaccountedTotal) + } else { + // There is an under-accounting at the moment, so we have to + // increase the memory reservation. + // + // This under-accounting can occur in a couple of edge cases: + // 1) the estimate of the response sizes is pretty good (i.e. + // respOverestimate is around 0), but we received many partial + // responses with ResumeSpans that take up much more space than + // the original requests; + // 2) we have a single large row in the response. In this case + // headOfLine must be true (targetBytes might be 1 or higher, + // but not enough for that large row). + toConsume := -overaccountedTotal + if err := w.s.budget.consume(ctx, toConsume, headOfLine /* allowDebt */); err != nil { + w.s.budget.release(ctx, targetBytes) + if !headOfLine { + // Since this is not the head of the line, we'll just + // discard the result and add the request back to be + // served. + // + // This is opportunistic behavior where we're hoping + // that once other requests are fully processed (i.e. + // the corresponding results are Release()'d), we'll be + // able to make progress on this request too. + // TODO(yuzefovich): consider updating the + // avgResponseSize and/or storing the information about + // the returned bytes size in req. + w.addRequest(req) + return + } + // The error indicates that the root memory pool has been + // exhausted, so we'll exit to be safe (in order not to OOM + // the node). + // TODO(yuzefovich): if the response contains multiple rows, + // consider adding the request back to be served with a note + // to issue it with smaller targetBytes. + w.s.setError(err) + return + } + } + // Update the resume request accordingly. + resumeReq.reqsReservedBytes = reqsMemUsage + + // Do admission control after we've finalized the memory accounting. + if br != nil && w.responseAdmissionQ != nil { + responseAdmission := admission.WorkInfo{ + TenantID: roachpb.SystemTenantID, + Priority: admission.WorkPriority(w.requestAdmissionHeader.Priority), + CreateTime: w.requestAdmissionHeader.CreateTime, + } + if _, err := w.responseAdmissionQ.Admit(ctx, responseAdmission); err != nil { + w.s.setError(err) + return + } + } + + // If we have any results, finalize them. + if len(results) > 0 { + w.finalizeSingleRangeResults( + results, memoryFootprintBytes, hasNonEmptyScanResponse, + numCompleteGetResponses, + ) + } + + // If we have any incomplete requests, add them back into the work + // pool. + if len(resumeReq.reqs) > 0 { + w.addRequest(resumeReq) + } + }); err != nil { + // The new goroutine for the request wasn't spun up, so we have to + // perform the cleanup of this request ourselves. + w.asyncRequestCleanup() + w.s.setError(err) + } +} + +// finalizeSingleRangeResults "finalizes" the results of evaluation of a +// singleRangeBatch. By "finalization" we mean setting Complete field of +// ScanResp to correct value for all scan responses, updating the estimate of an +// average response size, and telling the Streamer about these results. +// +// This method assumes that results has length greater than zero. +func (w *workerCoordinator) finalizeSingleRangeResults( + results []Result, + actualMemoryReservation int64, + hasNonEmptyScanResponse bool, + numCompleteGetResponses int, +) { + w.s.mu.Lock() + defer w.s.mu.Unlock() + + numCompleteResponses := numCompleteGetResponses + // If we have non-empty scan response, it might be complete. This will be + // the case when a scan response doesn't have a resume span and there are no + // other scan requests in flight (involving other ranges) that are part of + // the same original ScanRequest. + // + // We need to do this check as well as adding the results to be returned to + // the client as an atomic operation so that Complete is set to true only on + // the last partial scan response. + if hasNonEmptyScanResponse { + for _, r := range results { + if r.ScanResp.ScanResponse != nil { + if r.ScanResp.ResumeSpan == nil { + // The scan within the range is complete. + w.s.mu.numRangesLeftPerScanRequest[r.position]-- + if w.s.mu.numRangesLeftPerScanRequest[r.position] == 0 { + // The scan across all ranges is now complete too. + r.ScanResp.Complete = true + numCompleteResponses++ + } + } else { + // Unset the ResumeSpan on the result in order to not + // confuse the user of the Streamer. Non-nil resume span was + // already included into resumeReq populated in + // performRequestAsync. + r.ScanResp.ResumeSpan = nil + } + } + } + } + + // Update the average response size based on this batch. + // TODO(yuzefovich): some of the responses might be partial, yet the + // estimator doesn't distinguish the footprint of the full response vs the + // partial one. Think more about this. + w.s.mu.avgResponseEstimator.update(actualMemoryReservation, int64(len(results))) + w.s.mu.numCompleteRequests += numCompleteResponses + // Store the results and non-blockingly notify the Streamer about them. + w.s.mu.results = append(w.s.mu.results, results...) + w.s.notifyGetResultsLocked() +} + +var zeroIntSlice []int + +func init() { + zeroIntSlice = make([]int, 1<<10) +} + +const requestUnionSliceOverhead = int64(unsafe.Sizeof([]roachpb.RequestUnion{})) + +func requestsMemUsage(reqs []roachpb.RequestUnion) int64 { + memUsage := requestUnionSliceOverhead + // Slice up to the capacity to account for everything. + for _, r := range reqs[:cap(reqs)] { + memUsage += int64(r.Size()) + } + return memUsage +} diff --git a/pkg/kv/kvclient/kvstreamer/streamer_test.go b/pkg/kv/kvclient/kvstreamer/streamer_test.go new file mode 100644 index 000000000000..2183cbbd0f42 --- /dev/null +++ b/pkg/kv/kvclient/kvstreamer/streamer_test.go @@ -0,0 +1,271 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvstreamer + +import ( + "context" + "math" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/stretchr/testify/require" +) + +func getStreamer( + ctx context.Context, s serverutils.TestServerInterface, limitBytes int64, acc *mon.BoundAccount, +) *Streamer { + return NewStreamer( + s.DistSenderI().(*kvcoord.DistSender), + s.Stopper(), + kv.NewTxn(ctx, s.DB(), s.NodeID()), + cluster.MakeTestingClusterSettings(), + lock.WaitPolicy(0), + limitBytes, + acc, + ) +} + +// TestStreamerLimitations verifies that the streamer panics or encounters +// errors in currently unsupported or invalid scenarios. +func TestStreamerLimitations(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + getStreamer := func() *Streamer { + return getStreamer(ctx, s, math.MaxInt64, nil /* acc */) + } + + t.Run("InOrder mode unsupported", func(t *testing.T) { + require.Panics(t, func() { + streamer := getStreamer() + streamer.Init(InOrder, Hints{UniqueRequests: true}) + }) + }) + + t.Run("non-unique requests unsupported", func(t *testing.T) { + require.Panics(t, func() { + streamer := getStreamer() + streamer.Init(OutOfOrder, Hints{UniqueRequests: false}) + }) + }) + + t.Run("invalid enqueueKeys", func(t *testing.T) { + streamer := getStreamer() + defer streamer.Close() + streamer.Init(OutOfOrder, Hints{UniqueRequests: true}) + // Use a single request but two keys which is invalid. + reqs := []roachpb.RequestUnion{{Value: &roachpb.RequestUnion_Get{}}} + enqueueKeys := []int{0, 1} + require.Error(t, streamer.Enqueue(ctx, reqs, enqueueKeys)) + }) + + t.Run("pipelining unsupported", func(t *testing.T) { + streamer := getStreamer() + defer streamer.Close() + streamer.Init(OutOfOrder, Hints{UniqueRequests: true}) + get := roachpb.NewGet(roachpb.Key("key"), false /* forUpdate */) + reqs := []roachpb.RequestUnion{{ + Value: &roachpb.RequestUnion_Get{ + Get: get.(*roachpb.GetRequest), + }, + }} + require.NoError(t, streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */)) + // It is invalid to enqueue more requests before the previous have been + // responded to. + require.Error(t, streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */)) + }) +} + +// TestLargeKeys verifies that the Streamer successfully completes the queries +// when the keys to lookup are large (i.e. the enqueued requests themselves have +// large memory footprint). +func TestLargeKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStress(t, "the test inserts large blobs, and the machine can be overloaded when under stress") + + rng, _ := randutil.NewTestRand() + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + ctx := context.Background() + defer s.Stopper().Stop(ctx) + + // Lower the distsql_workmem limit so that we can operate with smaller + // blobs. Note that the joinReader in the row-by-row engine will override + // the limit if it is lower than 8MiB, so we cannot go lower than that here. + _, err := db.Exec("SET distsql_workmem='8MiB'") + require.NoError(t, err) + // In both engines, the index joiner buffers input rows up to 4MiB in size, + // so we have a couple of interesting options for the blob size: + // - 3000000 is interesting because it doesn't exceed the buffer size, yet + // two rows with such blobs do exceed it. The index joiners are expected to + // to process each row on its own. + // - 5000000 is interesting because a single row already exceeds the buffer + // size. + for _, blobSize := range []int{3000000, 5000000} { + // onlyLarge determines whether only large blobs are inserted or a mix + // of large and small blobs. + for _, onlyLarge := range []bool{false, true} { + _, err = db.Exec("DROP TABLE IF EXISTS foo") + require.NoError(t, err) + // We set up such a table that contains two large columns, one of them + // being the primary key. The idea is that the query below will first + // read from the secondary index which would include only the PK blob, + // and that will be used to construct index join lookups (i.e. the PK + // blobs will be the enqueued requests for the Streamer) whereas the + // other blob will be part of the response. + _, err = db.Exec("CREATE TABLE foo (pk_blob STRING PRIMARY KEY, attribute INT, blob TEXT, INDEX(attribute))") + require.NoError(t, err) + + // Insert a handful of rows. + numRows := rng.Intn(3) + 3 + for i := 0; i < numRows; i++ { + letter := string(byte('a') + byte(i)) + valueSize := blobSize + if !onlyLarge && rng.Float64() < 0.5 { + // If we're using a mix of large and small values, with 50% + // use a small value now. + valueSize = rng.Intn(10) + 1 + } + _, err = db.Exec("INSERT INTO foo SELECT repeat($1, $2), 1, repeat($1, $2)", letter, valueSize) + require.NoError(t, err) + } + + // Perform an index join so that the Streamer API is used. + query := "SELECT * FROM foo@foo_attribute_idx WHERE attribute=1" + testutils.RunTrueAndFalse(t, "vectorize", func(t *testing.T, vectorize bool) { + vectorizeMode := "off" + if vectorize { + vectorizeMode = "on" + } + _, err = db.Exec("SET vectorize = " + vectorizeMode) + require.NoError(t, err) + _, err = db.Exec(query) + require.NoError(t, err) + }) + } + } +} + +// TestStreamerBudgetErrorInEnqueue verifies the behavior of the Streamer in +// Enqueue when its limit and/or root pool limit are exceeded. Additional tests +// around the memory limit errors (when the responses exceed the limit) can be +// found in TestMemoryLimit in pkg/sql. +func TestStreamerBudgetErrorInEnqueue(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + // Create a dummy table for which we know the encoding of valid keys. + _, err := db.Exec("CREATE TABLE foo (pk_blob STRING PRIMARY KEY, attribute INT, blob TEXT, INDEX(attribute))") + require.NoError(t, err) + + // makeGetRequest returns a valid GetRequest that wants to lookup a key with + // value 'a' repeated keySize number of times in the primary index of table + // foo. + makeGetRequest := func(keySize int) roachpb.RequestUnion { + var res roachpb.RequestUnion + var get roachpb.GetRequest + var union roachpb.RequestUnion_Get + key := make([]byte, keySize+6) + key[0] = 190 + key[1] = 137 + key[2] = 18 + for i := 0; i < keySize; i++ { + key[i+3] = 97 + } + key[keySize+3] = 0 + key[keySize+4] = 1 + key[keySize+5] = 136 + get.Key = key + union.Get = &get + res.Value = &union + return res + } + + // Imitate a root SQL memory monitor with 1MiB size. + const rootPoolSize = 1 << 20 /* 1MiB */ + rootMemMonitor := mon.NewMonitor( + "root", /* name */ + mon.MemoryResource, + nil, /* curCount */ + nil, /* maxHist */ + -1, /* increment */ + math.MaxInt64, /* noteworthy */ + cluster.MakeTestingClusterSettings(), + ) + rootMemMonitor.Start(ctx, nil /* pool */, mon.MakeStandaloneBudget(rootPoolSize)) + defer rootMemMonitor.Stop(ctx) + + acc := rootMemMonitor.MakeBoundAccount() + defer acc.Close(ctx) + + getStreamer := func(limitBytes int64) *Streamer { + acc.Clear(ctx) + s := getStreamer(ctx, s, limitBytes, &acc) + s.Init(OutOfOrder, Hints{UniqueRequests: true}) + return s + } + + t.Run("single key exceeds limit", func(t *testing.T) { + const limitBytes = 10 + streamer := getStreamer(limitBytes) + defer streamer.Close() + + // A single request that exceeds the limit should be allowed. + reqs := make([]roachpb.RequestUnion, 1) + reqs[0] = makeGetRequest(limitBytes + 1) + require.NoError(t, streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */)) + }) + + t.Run("single key exceeds root pool size", func(t *testing.T) { + const limitBytes = 10 + streamer := getStreamer(limitBytes) + defer streamer.Close() + + // A single request that exceeds the limit as well as the root SQL pool + // should be denied. + reqs := make([]roachpb.RequestUnion, 1) + reqs[0] = makeGetRequest(rootPoolSize + 1) + require.Error(t, streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */)) + }) + + t.Run("multiple keys exceed limit", func(t *testing.T) { + const limitBytes = 10 + streamer := getStreamer(limitBytes) + defer streamer.Close() + + // Create two requests which exceed the limit when combined. + reqs := make([]roachpb.RequestUnion, 2) + reqs[0] = makeGetRequest(limitBytes/2 + 1) + reqs[1] = makeGetRequest(limitBytes/2 + 1) + require.Error(t, streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */)) + }) +} diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 15771b9f7ffd..474acd4c4b53 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -572,6 +572,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { ExternalStorage: cfg.externalStorage, ExternalStorageFromURI: cfg.externalStorageFromURI, + DistSender: cfg.distSender, RangeCache: cfg.distSender.RangeDescriptorCache(), SQLSQLResponseAdmissionQ: cfg.sqlSQLResponseAdmissionQ, CollectionFactory: collectionFactory, diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index ca19413d48e2..2167b94c5332 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -171,7 +171,7 @@ func (p *planner) AlterPrimaryKey( // If the new index is requested to be sharded, set up the index descriptor // to be sharded, and add the new shard column if it is missing. if alterPKNode.Sharded != nil { - shardCol, newColumns, newColumn, err := setupShardedIndex( + shardCol, newColumns, err := setupShardedIndex( ctx, p.EvalContext(), &p.semaCtx, @@ -186,15 +186,13 @@ func (p *planner) AlterPrimaryKey( return err } alterPKNode.Columns = newColumns - if newColumn { - if err := p.setupConstraintForShard( - ctx, - tableDesc, - shardCol, - newPrimaryIndexDesc.Sharded.ShardBuckets, - ); err != nil { - return err - } + if err := p.maybeSetupConstraintForShard( + ctx, + tableDesc, + shardCol, + newPrimaryIndexDesc.Sharded.ShardBuckets, + ); err != nil { + return err } telemetry.Inc(sqltelemetry.HashShardedIndexCounter) } diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 5f4481c07c7c..bfedbe738624 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -518,11 +518,6 @@ CREATE TABLE system.statement_statistics ( metadata, statistics, plan - ), - CONSTRAINT check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 CHECK ( - crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 IN ( - 0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8 - ) ) ) ` @@ -554,11 +549,6 @@ CREATE TABLE system.transaction_statistics ( agg_interval, metadata, statistics - ), - CONSTRAINT check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 CHECK ( - crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 IN ( - 0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8 - ) ) ); ` diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index c3835f5c7b62..ae37e3eb09ff 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -176,7 +176,7 @@ func supportedNatively(spec *execinfrapb.ProcessorSpec) error { return nil case spec.Core.JoinReader != nil: - if spec.Core.JoinReader.LookupColumns != nil || !spec.Core.JoinReader.LookupExpr.Empty() { + if !spec.Core.JoinReader.IsIndexJoin() { return errLookupJoinUnsupported } return nil @@ -740,7 +740,7 @@ func NewColOperator( if err := checkNumIn(inputs, 1); err != nil { return r, err } - if core.JoinReader.LookupColumns != nil || !core.JoinReader.LookupExpr.Empty() { + if !core.JoinReader.IsIndexJoin() { return r, errors.AssertionFailedf("lookup join reader is unsupported in vectorized") } // We have to create a separate account in order for the cFetcher to @@ -753,11 +753,23 @@ func NewColOperator( kvFetcherMemAcc := args.MonitorRegistry.CreateUnlimitedMemAccount( ctx, flowCtx, "kvfetcher" /* opName */, spec.ProcessorID, ) + var streamerBudgetAcc *mon.BoundAccount + // We have an index join, and when the ordering doesn't have to be + // maintained, we might use the Streamer API which requires a + // separate memory account that is bound to an unlimited memory + // monitor. + if !core.JoinReader.MaintainOrdering { + streamerBudgetAcc = args.MonitorRegistry.CreateUnlimitedMemAccount( + ctx, flowCtx, "streamer" /* opName */, spec.ProcessorID, + ) + } inputTypes := make([]*types.T, len(spec.Input[0].ColumnTypes)) copy(inputTypes, spec.Input[0].ColumnTypes) indexJoinOp, err := colfetcher.NewColIndexJoin( - ctx, getStreamingAllocator(ctx, args), colmem.NewAllocator(ctx, cFetcherMemAcc, factory), kvFetcherMemAcc, - flowCtx, args.ExprHelper, inputs[0].Root, core.JoinReader, post, inputTypes, + ctx, getStreamingAllocator(ctx, args), + colmem.NewAllocator(ctx, cFetcherMemAcc, factory), + kvFetcherMemAcc, streamerBudgetAcc, flowCtx, args.ExprHelper, + inputs[0].Root, core.JoinReader, post, inputTypes, ) if err != nil { return r, err diff --git a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go index 9dcfb07cd241..b3e122caaf83 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go @@ -50,12 +50,6 @@ type projBitandInt16ConstInt16Op struct { } func (p projBitandInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -152,12 +146,6 @@ type projBitandInt16ConstInt32Op struct { } func (p projBitandInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -254,12 +242,6 @@ type projBitandInt16ConstInt64Op struct { } func (p projBitandInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -356,12 +338,6 @@ type projBitandInt32ConstInt16Op struct { } func (p projBitandInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -458,12 +434,6 @@ type projBitandInt32ConstInt32Op struct { } func (p projBitandInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -560,12 +530,6 @@ type projBitandInt32ConstInt64Op struct { } func (p projBitandInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -662,12 +626,6 @@ type projBitandInt64ConstInt16Op struct { } func (p projBitandInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -764,12 +722,6 @@ type projBitandInt64ConstInt32Op struct { } func (p projBitandInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -866,12 +818,6 @@ type projBitandInt64ConstInt64Op struct { } func (p projBitandInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -964,16 +910,15 @@ func (p projBitandInt64ConstInt64Op) Next() coldata.Batch { type projBitandDatumConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitandDatumConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1096,12 +1041,6 @@ type projBitorInt16ConstInt16Op struct { } func (p projBitorInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1198,12 +1137,6 @@ type projBitorInt16ConstInt32Op struct { } func (p projBitorInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1300,12 +1233,6 @@ type projBitorInt16ConstInt64Op struct { } func (p projBitorInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1402,12 +1329,6 @@ type projBitorInt32ConstInt16Op struct { } func (p projBitorInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1504,12 +1425,6 @@ type projBitorInt32ConstInt32Op struct { } func (p projBitorInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1606,12 +1521,6 @@ type projBitorInt32ConstInt64Op struct { } func (p projBitorInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1708,12 +1617,6 @@ type projBitorInt64ConstInt16Op struct { } func (p projBitorInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1810,12 +1713,6 @@ type projBitorInt64ConstInt32Op struct { } func (p projBitorInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1912,12 +1809,6 @@ type projBitorInt64ConstInt64Op struct { } func (p projBitorInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2010,16 +1901,15 @@ func (p projBitorInt64ConstInt64Op) Next() coldata.Batch { type projBitorDatumConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitorDatumConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2142,12 +2032,6 @@ type projBitxorInt16ConstInt16Op struct { } func (p projBitxorInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2244,12 +2128,6 @@ type projBitxorInt16ConstInt32Op struct { } func (p projBitxorInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2346,12 +2224,6 @@ type projBitxorInt16ConstInt64Op struct { } func (p projBitxorInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2448,12 +2320,6 @@ type projBitxorInt32ConstInt16Op struct { } func (p projBitxorInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2550,12 +2416,6 @@ type projBitxorInt32ConstInt32Op struct { } func (p projBitxorInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2652,12 +2512,6 @@ type projBitxorInt32ConstInt64Op struct { } func (p projBitxorInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2754,12 +2608,6 @@ type projBitxorInt64ConstInt16Op struct { } func (p projBitxorInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2856,12 +2704,6 @@ type projBitxorInt64ConstInt32Op struct { } func (p projBitxorInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2958,12 +2800,6 @@ type projBitxorInt64ConstInt64Op struct { } func (p projBitxorInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3056,16 +2892,15 @@ func (p projBitxorInt64ConstInt64Op) Next() coldata.Batch { type projBitxorDatumConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitxorDatumConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3188,12 +3023,6 @@ type projPlusDecimalConstInt16Op struct { } func (p projPlusDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3318,12 +3147,6 @@ type projPlusDecimalConstInt32Op struct { } func (p projPlusDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3448,12 +3271,6 @@ type projPlusDecimalConstInt64Op struct { } func (p projPlusDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3578,12 +3395,6 @@ type projPlusDecimalConstDecimalOp struct { } func (p projPlusDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3704,12 +3515,6 @@ type projPlusInt16ConstInt16Op struct { } func (p projPlusInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3830,12 +3635,6 @@ type projPlusInt16ConstInt32Op struct { } func (p projPlusInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3956,12 +3755,6 @@ type projPlusInt16ConstInt64Op struct { } func (p projPlusInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4082,12 +3875,6 @@ type projPlusInt16ConstDecimalOp struct { } func (p projPlusInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4212,16 +3999,15 @@ func (p projPlusInt16ConstDecimalOp) Next() coldata.Batch { type projPlusInt16ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projPlusInt16ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4362,12 +4148,6 @@ type projPlusInt32ConstInt16Op struct { } func (p projPlusInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4488,12 +4268,6 @@ type projPlusInt32ConstInt32Op struct { } func (p projPlusInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4614,12 +4388,6 @@ type projPlusInt32ConstInt64Op struct { } func (p projPlusInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4740,12 +4508,6 @@ type projPlusInt32ConstDecimalOp struct { } func (p projPlusInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4870,16 +4632,15 @@ func (p projPlusInt32ConstDecimalOp) Next() coldata.Batch { type projPlusInt32ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projPlusInt32ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5020,12 +4781,6 @@ type projPlusInt64ConstInt16Op struct { } func (p projPlusInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5146,12 +4901,6 @@ type projPlusInt64ConstInt32Op struct { } func (p projPlusInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5272,12 +5021,6 @@ type projPlusInt64ConstInt64Op struct { } func (p projPlusInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5398,12 +5141,6 @@ type projPlusInt64ConstDecimalOp struct { } func (p projPlusInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5528,16 +5265,15 @@ func (p projPlusInt64ConstDecimalOp) Next() coldata.Batch { type projPlusInt64ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projPlusInt64ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5678,12 +5414,6 @@ type projPlusFloat64ConstFloat64Op struct { } func (p projPlusFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5792,12 +5522,6 @@ type projPlusTimestampConstIntervalOp struct { } func (p projPlusTimestampConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5906,12 +5630,6 @@ type projPlusIntervalConstTimestampOp struct { } func (p projPlusIntervalConstTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6020,12 +5738,6 @@ type projPlusIntervalConstIntervalOp struct { } func (p projPlusIntervalConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6110,16 +5822,15 @@ func (p projPlusIntervalConstIntervalOp) Next() coldata.Batch { type projPlusIntervalConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg duration.Duration } func (p projPlusIntervalConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6256,16 +5967,15 @@ func (p projPlusIntervalConstDatumOp) Next() coldata.Batch { type projPlusDatumConstIntervalOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusDatumConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6400,16 +6110,15 @@ func (p projPlusDatumConstIntervalOp) Next() coldata.Batch { type projPlusDatumConstInt16Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusDatumConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6544,16 +6253,15 @@ func (p projPlusDatumConstInt16Op) Next() coldata.Batch { type projPlusDatumConstInt32Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusDatumConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6688,16 +6396,15 @@ func (p projPlusDatumConstInt32Op) Next() coldata.Batch { type projPlusDatumConstInt64Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusDatumConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6836,12 +6543,6 @@ type projMinusDecimalConstInt16Op struct { } func (p projMinusDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6966,12 +6667,6 @@ type projMinusDecimalConstInt32Op struct { } func (p projMinusDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7096,12 +6791,6 @@ type projMinusDecimalConstInt64Op struct { } func (p projMinusDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7226,12 +6915,6 @@ type projMinusDecimalConstDecimalOp struct { } func (p projMinusDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7352,12 +7035,6 @@ type projMinusInt16ConstInt16Op struct { } func (p projMinusInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7478,12 +7155,6 @@ type projMinusInt16ConstInt32Op struct { } func (p projMinusInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7604,12 +7275,6 @@ type projMinusInt16ConstInt64Op struct { } func (p projMinusInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7730,12 +7395,6 @@ type projMinusInt16ConstDecimalOp struct { } func (p projMinusInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7860,16 +7519,15 @@ func (p projMinusInt16ConstDecimalOp) Next() coldata.Batch { type projMinusInt16ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projMinusInt16ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8010,12 +7668,6 @@ type projMinusInt32ConstInt16Op struct { } func (p projMinusInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8136,12 +7788,6 @@ type projMinusInt32ConstInt32Op struct { } func (p projMinusInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8262,12 +7908,6 @@ type projMinusInt32ConstInt64Op struct { } func (p projMinusInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8388,12 +8028,6 @@ type projMinusInt32ConstDecimalOp struct { } func (p projMinusInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8518,16 +8152,15 @@ func (p projMinusInt32ConstDecimalOp) Next() coldata.Batch { type projMinusInt32ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projMinusInt32ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8668,12 +8301,6 @@ type projMinusInt64ConstInt16Op struct { } func (p projMinusInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8794,12 +8421,6 @@ type projMinusInt64ConstInt32Op struct { } func (p projMinusInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8920,12 +8541,6 @@ type projMinusInt64ConstInt64Op struct { } func (p projMinusInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9046,12 +8661,6 @@ type projMinusInt64ConstDecimalOp struct { } func (p projMinusInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9176,16 +8785,15 @@ func (p projMinusInt64ConstDecimalOp) Next() coldata.Batch { type projMinusInt64ConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projMinusInt64ConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9326,12 +8934,6 @@ type projMinusFloat64ConstFloat64Op struct { } func (p projMinusFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9440,12 +9042,6 @@ type projMinusTimestampConstTimestampOp struct { } func (p projMinusTimestampConstTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9546,12 +9142,6 @@ type projMinusTimestampConstIntervalOp struct { } func (p projMinusTimestampConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9660,12 +9250,6 @@ type projMinusIntervalConstIntervalOp struct { } func (p projMinusIntervalConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9750,16 +9334,15 @@ func (p projMinusIntervalConstIntervalOp) Next() coldata.Batch { type projMinusIntervalConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg duration.Duration } func (p projMinusIntervalConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9900,12 +9483,6 @@ type projMinusJSONConstBytesOp struct { } func (p projMinusJSONConstBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10028,12 +9605,6 @@ type projMinusJSONConstInt16Op struct { } func (p projMinusJSONConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10140,12 +9711,6 @@ type projMinusJSONConstInt32Op struct { } func (p projMinusJSONConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10252,12 +9817,6 @@ type projMinusJSONConstInt64Op struct { } func (p projMinusJSONConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10360,16 +9919,15 @@ func (p projMinusJSONConstInt64Op) Next() coldata.Batch { type projMinusDatumConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10488,16 +10046,15 @@ func (p projMinusDatumConstDatumOp) Next() coldata.Batch { type projMinusDatumConstIntervalOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10632,16 +10189,15 @@ func (p projMinusDatumConstIntervalOp) Next() coldata.Batch { type projMinusDatumConstBytesOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10776,16 +10332,15 @@ func (p projMinusDatumConstBytesOp) Next() coldata.Batch { type projMinusDatumConstInt16Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10920,16 +10475,15 @@ func (p projMinusDatumConstInt16Op) Next() coldata.Batch { type projMinusDatumConstInt32Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11064,16 +10618,15 @@ func (p projMinusDatumConstInt32Op) Next() coldata.Batch { type projMinusDatumConstInt64Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11212,12 +10765,6 @@ type projMultDecimalConstInt16Op struct { } func (p projMultDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11342,12 +10889,6 @@ type projMultDecimalConstInt32Op struct { } func (p projMultDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11472,12 +11013,6 @@ type projMultDecimalConstInt64Op struct { } func (p projMultDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11602,12 +11137,6 @@ type projMultDecimalConstDecimalOp struct { } func (p projMultDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11728,12 +11257,6 @@ type projMultDecimalConstIntervalOp struct { } func (p projMultDecimalConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11842,12 +11365,6 @@ type projMultInt16ConstInt16Op struct { } func (p projMultInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12000,12 +11517,6 @@ type projMultInt16ConstInt32Op struct { } func (p projMultInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12158,12 +11669,6 @@ type projMultInt16ConstInt64Op struct { } func (p projMultInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12316,12 +11821,6 @@ type projMultInt16ConstDecimalOp struct { } func (p projMultInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12450,12 +11949,6 @@ type projMultInt16ConstIntervalOp struct { } func (p projMultInt16ConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12544,12 +12037,6 @@ type projMultInt32ConstInt16Op struct { } func (p projMultInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12702,12 +12189,6 @@ type projMultInt32ConstInt32Op struct { } func (p projMultInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12860,12 +12341,6 @@ type projMultInt32ConstInt64Op struct { } func (p projMultInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13018,12 +12493,6 @@ type projMultInt32ConstDecimalOp struct { } func (p projMultInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13152,12 +12621,6 @@ type projMultInt32ConstIntervalOp struct { } func (p projMultInt32ConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13246,12 +12709,6 @@ type projMultInt64ConstInt16Op struct { } func (p projMultInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13404,12 +12861,6 @@ type projMultInt64ConstInt32Op struct { } func (p projMultInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13562,12 +13013,6 @@ type projMultInt64ConstInt64Op struct { } func (p projMultInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13720,12 +13165,6 @@ type projMultInt64ConstDecimalOp struct { } func (p projMultInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13854,12 +13293,6 @@ type projMultInt64ConstIntervalOp struct { } func (p projMultInt64ConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13948,12 +13381,6 @@ type projMultFloat64ConstFloat64Op struct { } func (p projMultFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14062,12 +13489,6 @@ type projMultFloat64ConstIntervalOp struct { } func (p projMultFloat64ConstIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14156,12 +13577,6 @@ type projMultIntervalConstInt16Op struct { } func (p projMultIntervalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14250,12 +13665,6 @@ type projMultIntervalConstInt32Op struct { } func (p projMultIntervalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14344,12 +13753,6 @@ type projMultIntervalConstInt64Op struct { } func (p projMultIntervalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14438,12 +13841,6 @@ type projMultIntervalConstFloat64Op struct { } func (p projMultIntervalConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14532,12 +13929,6 @@ type projMultIntervalConstDecimalOp struct { } func (p projMultIntervalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14646,12 +14037,6 @@ type projDivDecimalConstInt16Op struct { } func (p projDivDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14792,12 +14177,6 @@ type projDivDecimalConstInt32Op struct { } func (p projDivDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14938,12 +14317,6 @@ type projDivDecimalConstInt64Op struct { } func (p projDivDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15084,12 +14457,6 @@ type projDivDecimalConstDecimalOp struct { } func (p projDivDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15226,12 +14593,6 @@ type projDivInt16ConstInt16Op struct { } func (p projDivInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15368,12 +14729,6 @@ type projDivInt16ConstInt32Op struct { } func (p projDivInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15510,12 +14865,6 @@ type projDivInt16ConstInt64Op struct { } func (p projDivInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15652,12 +15001,6 @@ type projDivInt16ConstDecimalOp struct { } func (p projDivInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15802,12 +15145,6 @@ type projDivInt32ConstInt16Op struct { } func (p projDivInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15944,12 +15281,6 @@ type projDivInt32ConstInt32Op struct { } func (p projDivInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16086,12 +15417,6 @@ type projDivInt32ConstInt64Op struct { } func (p projDivInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16228,12 +15553,6 @@ type projDivInt32ConstDecimalOp struct { } func (p projDivInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16378,12 +15697,6 @@ type projDivInt64ConstInt16Op struct { } func (p projDivInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16520,12 +15833,6 @@ type projDivInt64ConstInt32Op struct { } func (p projDivInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16662,12 +15969,6 @@ type projDivInt64ConstInt64Op struct { } func (p projDivInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16804,12 +16105,6 @@ type projDivInt64ConstDecimalOp struct { } func (p projDivInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16954,12 +16249,6 @@ type projDivFloat64ConstFloat64Op struct { } func (p projDivFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17084,12 +16373,6 @@ type projDivIntervalConstInt64Op struct { } func (p projDivIntervalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17194,12 +16477,6 @@ type projDivIntervalConstFloat64Op struct { } func (p projDivIntervalConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17304,12 +16581,6 @@ type projFloorDivDecimalConstInt16Op struct { } func (p projFloorDivDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17450,12 +16721,6 @@ type projFloorDivDecimalConstInt32Op struct { } func (p projFloorDivDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17596,12 +16861,6 @@ type projFloorDivDecimalConstInt64Op struct { } func (p projFloorDivDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17742,12 +17001,6 @@ type projFloorDivDecimalConstDecimalOp struct { } func (p projFloorDivDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17884,12 +17137,6 @@ type projFloorDivInt16ConstInt16Op struct { } func (p projFloorDivInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18006,12 +17253,6 @@ type projFloorDivInt16ConstInt32Op struct { } func (p projFloorDivInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18128,12 +17369,6 @@ type projFloorDivInt16ConstInt64Op struct { } func (p projFloorDivInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18250,12 +17485,6 @@ type projFloorDivInt16ConstDecimalOp struct { } func (p projFloorDivInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18400,12 +17629,6 @@ type projFloorDivInt32ConstInt16Op struct { } func (p projFloorDivInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18522,12 +17745,6 @@ type projFloorDivInt32ConstInt32Op struct { } func (p projFloorDivInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18644,12 +17861,6 @@ type projFloorDivInt32ConstInt64Op struct { } func (p projFloorDivInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18766,12 +17977,6 @@ type projFloorDivInt32ConstDecimalOp struct { } func (p projFloorDivInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18916,12 +18121,6 @@ type projFloorDivInt64ConstInt16Op struct { } func (p projFloorDivInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19038,12 +18237,6 @@ type projFloorDivInt64ConstInt32Op struct { } func (p projFloorDivInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19160,12 +18353,6 @@ type projFloorDivInt64ConstInt64Op struct { } func (p projFloorDivInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19282,12 +18469,6 @@ type projFloorDivInt64ConstDecimalOp struct { } func (p projFloorDivInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19432,12 +18613,6 @@ type projFloorDivFloat64ConstFloat64Op struct { } func (p projFloorDivFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19562,12 +18737,6 @@ type projModDecimalConstInt16Op struct { } func (p projModDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19708,12 +18877,6 @@ type projModDecimalConstInt32Op struct { } func (p projModDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19854,12 +19017,6 @@ type projModDecimalConstInt64Op struct { } func (p projModDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20000,12 +19157,6 @@ type projModDecimalConstDecimalOp struct { } func (p projModDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20142,12 +19293,6 @@ type projModInt16ConstInt16Op struct { } func (p projModInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20264,12 +19409,6 @@ type projModInt16ConstInt32Op struct { } func (p projModInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20386,12 +19525,6 @@ type projModInt16ConstInt64Op struct { } func (p projModInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20508,12 +19641,6 @@ type projModInt16ConstDecimalOp struct { } func (p projModInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20658,12 +19785,6 @@ type projModInt32ConstInt16Op struct { } func (p projModInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20780,12 +19901,6 @@ type projModInt32ConstInt32Op struct { } func (p projModInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20902,12 +20017,6 @@ type projModInt32ConstInt64Op struct { } func (p projModInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21024,12 +20133,6 @@ type projModInt32ConstDecimalOp struct { } func (p projModInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21174,12 +20277,6 @@ type projModInt64ConstInt16Op struct { } func (p projModInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21296,12 +20393,6 @@ type projModInt64ConstInt32Op struct { } func (p projModInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21418,12 +20509,6 @@ type projModInt64ConstInt64Op struct { } func (p projModInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21540,12 +20625,6 @@ type projModInt64ConstDecimalOp struct { } func (p projModInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21690,12 +20769,6 @@ type projModFloat64ConstFloat64Op struct { } func (p projModFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21820,12 +20893,6 @@ type projPowDecimalConstInt16Op struct { } func (p projPowDecimalConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21950,12 +21017,6 @@ type projPowDecimalConstInt32Op struct { } func (p projPowDecimalConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22080,12 +21141,6 @@ type projPowDecimalConstInt64Op struct { } func (p projPowDecimalConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22210,12 +21265,6 @@ type projPowDecimalConstDecimalOp struct { } func (p projPowDecimalConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22336,12 +21385,6 @@ type projPowInt16ConstInt16Op struct { } func (p projPowInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22486,12 +21529,6 @@ type projPowInt16ConstInt32Op struct { } func (p projPowInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22636,12 +21673,6 @@ type projPowInt16ConstInt64Op struct { } func (p projPowInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22786,12 +21817,6 @@ type projPowInt16ConstDecimalOp struct { } func (p projPowInt16ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22920,12 +21945,6 @@ type projPowInt32ConstInt16Op struct { } func (p projPowInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23070,12 +22089,6 @@ type projPowInt32ConstInt32Op struct { } func (p projPowInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23220,12 +22233,6 @@ type projPowInt32ConstInt64Op struct { } func (p projPowInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23370,12 +22377,6 @@ type projPowInt32ConstDecimalOp struct { } func (p projPowInt32ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23504,12 +22505,6 @@ type projPowInt64ConstInt16Op struct { } func (p projPowInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23654,12 +22649,6 @@ type projPowInt64ConstInt32Op struct { } func (p projPowInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23804,12 +22793,6 @@ type projPowInt64ConstInt64Op struct { } func (p projPowInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23954,12 +22937,6 @@ type projPowInt64ConstDecimalOp struct { } func (p projPowInt64ConstDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24088,12 +23065,6 @@ type projPowFloat64ConstFloat64Op struct { } func (p projPowFloat64ConstFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24202,12 +23173,6 @@ type projConcatBytesConstBytesOp struct { } func (p projConcatBytesConstBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24322,12 +23287,6 @@ type projConcatJSONConstJSONOp struct { } func (p projConcatJSONConstJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24434,16 +23393,15 @@ func (p projConcatJSONConstJSONOp) Next() coldata.Batch { type projConcatDatumConstDatumOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projConcatDatumConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24566,12 +23524,6 @@ type projLShiftInt16ConstInt16Op struct { } func (p projLShiftInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24692,12 +23644,6 @@ type projLShiftInt16ConstInt32Op struct { } func (p projLShiftInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24818,12 +23764,6 @@ type projLShiftInt16ConstInt64Op struct { } func (p projLShiftInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24944,12 +23884,6 @@ type projLShiftInt32ConstInt16Op struct { } func (p projLShiftInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25070,12 +24004,6 @@ type projLShiftInt32ConstInt32Op struct { } func (p projLShiftInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25196,12 +24124,6 @@ type projLShiftInt32ConstInt64Op struct { } func (p projLShiftInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25322,12 +24244,6 @@ type projLShiftInt64ConstInt16Op struct { } func (p projLShiftInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25448,12 +24364,6 @@ type projLShiftInt64ConstInt32Op struct { } func (p projLShiftInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25574,12 +24484,6 @@ type projLShiftInt64ConstInt64Op struct { } func (p projLShiftInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25696,16 +24600,15 @@ func (p projLShiftInt64ConstInt64Op) Next() coldata.Batch { type projLShiftDatumConstInt16Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projLShiftDatumConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25840,16 +24743,15 @@ func (p projLShiftDatumConstInt16Op) Next() coldata.Batch { type projLShiftDatumConstInt32Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projLShiftDatumConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25984,16 +24886,15 @@ func (p projLShiftDatumConstInt32Op) Next() coldata.Batch { type projLShiftDatumConstInt64Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projLShiftDatumConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26132,12 +25033,6 @@ type projRShiftInt16ConstInt16Op struct { } func (p projRShiftInt16ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26258,12 +25153,6 @@ type projRShiftInt16ConstInt32Op struct { } func (p projRShiftInt16ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26384,12 +25273,6 @@ type projRShiftInt16ConstInt64Op struct { } func (p projRShiftInt16ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26510,12 +25393,6 @@ type projRShiftInt32ConstInt16Op struct { } func (p projRShiftInt32ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26636,12 +25513,6 @@ type projRShiftInt32ConstInt32Op struct { } func (p projRShiftInt32ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26762,12 +25633,6 @@ type projRShiftInt32ConstInt64Op struct { } func (p projRShiftInt32ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26888,12 +25753,6 @@ type projRShiftInt64ConstInt16Op struct { } func (p projRShiftInt64ConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27014,12 +25873,6 @@ type projRShiftInt64ConstInt32Op struct { } func (p projRShiftInt64ConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27140,12 +25993,6 @@ type projRShiftInt64ConstInt64Op struct { } func (p projRShiftInt64ConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27262,16 +26109,15 @@ func (p projRShiftInt64ConstInt64Op) Next() coldata.Batch { type projRShiftDatumConstInt16Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projRShiftDatumConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27406,16 +26252,15 @@ func (p projRShiftDatumConstInt16Op) Next() coldata.Batch { type projRShiftDatumConstInt32Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projRShiftDatumConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27550,16 +26395,15 @@ func (p projRShiftDatumConstInt32Op) Next() coldata.Batch { type projRShiftDatumConstInt64Op struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projRShiftDatumConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27698,12 +26542,6 @@ type projJSONFetchValJSONConstBytesOp struct { } func (p projJSONFetchValJSONConstBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27838,12 +26676,6 @@ type projJSONFetchValJSONConstInt16Op struct { } func (p projJSONFetchValJSONConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27966,12 +26798,6 @@ type projJSONFetchValJSONConstInt32Op struct { } func (p projJSONFetchValJSONConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28094,12 +26920,6 @@ type projJSONFetchValJSONConstInt64Op struct { } func (p projJSONFetchValJSONConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28222,12 +27042,6 @@ type projJSONFetchTextJSONConstBytesOp struct { } func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28398,12 +27212,6 @@ type projJSONFetchTextJSONConstInt16Op struct { } func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28562,12 +27370,6 @@ type projJSONFetchTextJSONConstInt32Op struct { } func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28726,12 +27528,6 @@ type projJSONFetchTextJSONConstInt64Op struct { } func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28890,12 +27686,6 @@ type projJSONFetchValPathJSONConstDatumOp struct { } func (p projJSONFetchValPathJSONConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29018,12 +27808,6 @@ type projJSONFetchTextPathJSONConstDatumOp struct { } func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29202,7 +27986,6 @@ func GetProjectionLConstOperator( allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, - overloadHelper: execgen.OverloadHelper{BinFn: binFn, EvalCtx: evalCtx}, } c := colconv.GetDatumToPhysicalFn(constType)(constArg) leftType, rightType := constType, inputTypes[colIdx] @@ -29218,21 +28001,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt16ConstInt16Op{ + op := &projBitandInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitandInt16ConstInt32Op{ + op := &projBitandInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projBitandInt16ConstInt64Op{ + op := &projBitandInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -29240,21 +28026,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt32ConstInt16Op{ + op := &projBitandInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projBitandInt32ConstInt32Op{ + op := &projBitandInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitandInt32ConstInt64Op{ + op := &projBitandInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -29263,21 +28052,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt64ConstInt16Op{ + op := &projBitandInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projBitandInt64ConstInt32Op{ + op := &projBitandInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projBitandInt64ConstInt64Op{ + op := &projBitandInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -29290,10 +28082,12 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projBitandDatumConstDatumOp{ + op := &projBitandDatumConstDatumOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29307,21 +28101,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt16ConstInt16Op{ + op := &projBitorInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitorInt16ConstInt32Op{ + op := &projBitorInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projBitorInt16ConstInt64Op{ + op := &projBitorInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -29329,21 +28126,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt32ConstInt16Op{ + op := &projBitorInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projBitorInt32ConstInt32Op{ + op := &projBitorInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitorInt32ConstInt64Op{ + op := &projBitorInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -29352,21 +28152,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt64ConstInt16Op{ + op := &projBitorInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projBitorInt64ConstInt32Op{ + op := &projBitorInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projBitorInt64ConstInt64Op{ + op := &projBitorInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -29379,10 +28182,12 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projBitorDatumConstDatumOp{ + op := &projBitorDatumConstDatumOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29396,21 +28201,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt16ConstInt16Op{ + op := &projBitxorInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitxorInt16ConstInt32Op{ + op := &projBitxorInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt16ConstInt64Op{ + op := &projBitxorInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -29418,21 +28226,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt32ConstInt16Op{ + op := &projBitxorInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projBitxorInt32ConstInt32Op{ + op := &projBitxorInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt32ConstInt64Op{ + op := &projBitxorInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -29441,21 +28252,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt64ConstInt16Op{ + op := &projBitxorInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projBitxorInt64ConstInt32Op{ + op := &projBitxorInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt64ConstInt64Op{ + op := &projBitxorInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -29468,10 +28282,12 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projBitxorDatumConstDatumOp{ + op := &projBitxorDatumConstDatumOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29486,30 +28302,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDecimalConstInt16Op{ + op := &projPlusDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projPlusDecimalConstInt32Op{ + op := &projPlusDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projPlusDecimalConstInt64Op{ + op := &projPlusDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusDecimalConstDecimalOp{ + op := &projPlusDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -29520,39 +28340,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt16ConstInt16Op{ + op := &projPlusInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPlusInt16ConstInt32Op{ + op := &projPlusInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projPlusInt16ConstInt64Op{ + op := &projPlusInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16ConstDecimalOp{ + op := &projPlusInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16ConstDatumOp{ + op := &projPlusInt16ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -29560,39 +28386,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt32ConstInt16Op{ + op := &projPlusInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projPlusInt32ConstInt32Op{ + op := &projPlusInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPlusInt32ConstInt64Op{ + op := &projPlusInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32ConstDecimalOp{ + op := &projPlusInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32ConstDatumOp{ + op := &projPlusInt32ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -29601,39 +28433,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt64ConstInt16Op{ + op := &projPlusInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projPlusInt64ConstInt32Op{ + op := &projPlusInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projPlusInt64ConstInt64Op{ + op := &projPlusInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64ConstDecimalOp{ + op := &projPlusInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64ConstDatumOp{ + op := &projPlusInt64ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29646,10 +28484,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projPlusFloat64ConstFloat64Op{ + op := &projPlusFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -29662,10 +28501,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projPlusTimestampConstIntervalOp{ + op := &projPlusTimestampConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(time.Time), - }, nil + } + return op, nil } } } @@ -29678,28 +28518,32 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projPlusIntervalConstTimestampOp{ + op := &projPlusIntervalConstTimestampOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalConstIntervalOp{ + op := &projPlusIntervalConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalConstDatumOp{ + op := &projPlusIntervalConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29712,29 +28556,37 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projPlusDatumConstIntervalOp{ + op := &projPlusDatumConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDatumConstInt16Op{ + op := &projPlusDatumConstInt16Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projPlusDatumConstInt32Op{ + op := &projPlusDatumConstInt32Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projPlusDatumConstInt64Op{ + op := &projPlusDatumConstInt64Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29749,30 +28601,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDecimalConstInt16Op{ + op := &projMinusDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projMinusDecimalConstInt32Op{ + op := &projMinusDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projMinusDecimalConstInt64Op{ + op := &projMinusDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusDecimalConstDecimalOp{ + op := &projMinusDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -29783,39 +28639,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt16ConstInt16Op{ + op := &projMinusInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusInt16ConstInt32Op{ + op := &projMinusInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projMinusInt16ConstInt64Op{ + op := &projMinusInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16ConstDecimalOp{ + op := &projMinusInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16ConstDatumOp{ + op := &projMinusInt16ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -29823,39 +28685,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt32ConstInt16Op{ + op := &projMinusInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projMinusInt32ConstInt32Op{ + op := &projMinusInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusInt32ConstInt64Op{ + op := &projMinusInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32ConstDecimalOp{ + op := &projMinusInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32ConstDatumOp{ + op := &projMinusInt32ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -29864,39 +28732,45 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt64ConstInt16Op{ + op := &projMinusInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projMinusInt64ConstInt32Op{ + op := &projMinusInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projMinusInt64ConstInt64Op{ + op := &projMinusInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64ConstDecimalOp{ + op := &projMinusInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64ConstDatumOp{ + op := &projMinusInt64ConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29909,10 +28783,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMinusFloat64ConstFloat64Op{ + op := &projMinusFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -29925,19 +28800,21 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMinusTimestampConstTimestampOp{ + op := &projMinusTimestampConstTimestampOp{ projConstOpBase: projConstOpBase, constArg: c.(time.Time), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusTimestampConstIntervalOp{ + op := &projMinusTimestampConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(time.Time), - }, nil + } + return op, nil } } } @@ -29950,19 +28827,22 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMinusIntervalConstIntervalOp{ + op := &projMinusIntervalConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusIntervalConstDatumOp{ + op := &projMinusIntervalConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -29975,29 +28855,33 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMinusJSONConstBytesOp{ + op := &projMinusJSONConstBytesOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusJSONConstInt16Op{ + op := &projMinusJSONConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case 32: - return &projMinusJSONConstInt32Op{ + op := &projMinusJSONConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case -1: default: - return &projMinusJSONConstInt64Op{ + op := &projMinusJSONConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -30010,47 +28894,59 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMinusDatumConstDatumOp{ + op := &projMinusDatumConstDatumOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumConstIntervalOp{ + op := &projMinusDatumConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.BytesFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumConstBytesOp{ + op := &projMinusDatumConstBytesOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDatumConstInt16Op{ + op := &projMinusDatumConstInt16Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projMinusDatumConstInt32Op{ + op := &projMinusDatumConstInt32Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projMinusDatumConstInt64Op{ + op := &projMinusDatumConstInt64Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -30065,39 +28961,44 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultDecimalConstInt16Op{ + op := &projMultDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projMultDecimalConstInt32Op{ + op := &projMultDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projMultDecimalConstInt64Op{ + op := &projMultDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalConstDecimalOp{ + op := &projMultDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalConstIntervalOp{ + op := &projMultDecimalConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -30108,39 +29009,44 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt16ConstInt16Op{ + op := &projMultInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultInt16ConstInt32Op{ + op := &projMultInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projMultInt16ConstInt64Op{ + op := &projMultInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16ConstDecimalOp{ + op := &projMultInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16ConstIntervalOp{ + op := &projMultInt16ConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -30148,39 +29054,44 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt32ConstInt16Op{ + op := &projMultInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projMultInt32ConstInt32Op{ + op := &projMultInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultInt32ConstInt64Op{ + op := &projMultInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32ConstDecimalOp{ + op := &projMultInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32ConstIntervalOp{ + op := &projMultInt32ConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -30189,39 +29100,44 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt64ConstInt16Op{ + op := &projMultInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projMultInt64ConstInt32Op{ + op := &projMultInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projMultInt64ConstInt64Op{ + op := &projMultInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64ConstDecimalOp{ + op := &projMultInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64ConstIntervalOp{ + op := &projMultInt64ConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -30234,19 +29150,21 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projMultFloat64ConstFloat64Op{ + op := &projMultFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultFloat64ConstIntervalOp{ + op := &projMultFloat64ConstIntervalOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -30258,39 +29176,44 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultIntervalConstInt16Op{ + op := &projMultIntervalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil case 32: - return &projMultIntervalConstInt32Op{ + op := &projMultIntervalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil case -1: default: - return &projMultIntervalConstInt64Op{ + op := &projMultIntervalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalConstFloat64Op{ + op := &projMultIntervalConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalConstDecimalOp{ + op := &projMultIntervalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -30305,30 +29228,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivDecimalConstInt16Op{ + op := &projDivDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projDivDecimalConstInt32Op{ + op := &projDivDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projDivDecimalConstInt64Op{ + op := &projDivDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivDecimalConstDecimalOp{ + op := &projDivDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -30339,30 +29266,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt16ConstInt16Op{ + op := &projDivInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projDivInt16ConstInt32Op{ + op := &projDivInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projDivInt16ConstInt64Op{ + op := &projDivInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt16ConstDecimalOp{ + op := &projDivInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -30370,30 +29301,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt32ConstInt16Op{ + op := &projDivInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projDivInt32ConstInt32Op{ + op := &projDivInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projDivInt32ConstInt64Op{ + op := &projDivInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt32ConstDecimalOp{ + op := &projDivInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -30402,30 +29337,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt64ConstInt16Op{ + op := &projDivInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projDivInt64ConstInt32Op{ + op := &projDivInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projDivInt64ConstInt64Op{ + op := &projDivInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt64ConstDecimalOp{ + op := &projDivInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -30438,10 +29377,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projDivFloat64ConstFloat64Op{ + op := &projDivFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -30454,19 +29394,21 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projDivIntervalConstInt64Op{ + op := &projDivIntervalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projDivIntervalConstFloat64Op{ + op := &projDivIntervalConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -30481,30 +29423,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivDecimalConstInt16Op{ + op := &projFloorDivDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projFloorDivDecimalConstInt32Op{ + op := &projFloorDivDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projFloorDivDecimalConstInt64Op{ + op := &projFloorDivDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivDecimalConstDecimalOp{ + op := &projFloorDivDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -30515,30 +29461,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt16ConstInt16Op{ + op := &projFloorDivInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projFloorDivInt16ConstInt32Op{ + op := &projFloorDivInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt16ConstInt64Op{ + op := &projFloorDivInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt16ConstDecimalOp{ + op := &projFloorDivInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -30546,30 +29496,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt32ConstInt16Op{ + op := &projFloorDivInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projFloorDivInt32ConstInt32Op{ + op := &projFloorDivInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt32ConstInt64Op{ + op := &projFloorDivInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt32ConstDecimalOp{ + op := &projFloorDivInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -30578,30 +29532,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt64ConstInt16Op{ + op := &projFloorDivInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projFloorDivInt64ConstInt32Op{ + op := &projFloorDivInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt64ConstInt64Op{ + op := &projFloorDivInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt64ConstDecimalOp{ + op := &projFloorDivInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -30614,10 +29572,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projFloorDivFloat64ConstFloat64Op{ + op := &projFloorDivFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -30632,30 +29591,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModDecimalConstInt16Op{ + op := &projModDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projModDecimalConstInt32Op{ + op := &projModDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projModDecimalConstInt64Op{ + op := &projModDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModDecimalConstDecimalOp{ + op := &projModDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -30666,30 +29629,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt16ConstInt16Op{ + op := &projModInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projModInt16ConstInt32Op{ + op := &projModInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projModInt16ConstInt64Op{ + op := &projModInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt16ConstDecimalOp{ + op := &projModInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -30697,30 +29664,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt32ConstInt16Op{ + op := &projModInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projModInt32ConstInt32Op{ + op := &projModInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projModInt32ConstInt64Op{ + op := &projModInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt32ConstDecimalOp{ + op := &projModInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -30729,30 +29700,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt64ConstInt16Op{ + op := &projModInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projModInt64ConstInt32Op{ + op := &projModInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projModInt64ConstInt64Op{ + op := &projModInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt64ConstDecimalOp{ + op := &projModInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -30765,10 +29740,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projModFloat64ConstFloat64Op{ + op := &projModFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -30783,30 +29759,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowDecimalConstInt16Op{ + op := &projPowDecimalConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case 32: - return &projPowDecimalConstInt32Op{ + op := &projPowDecimalConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil case -1: default: - return &projPowDecimalConstInt64Op{ + op := &projPowDecimalConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowDecimalConstDecimalOp{ + op := &projPowDecimalConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -30817,30 +29797,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt16ConstInt16Op{ + op := &projPowInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPowInt16ConstInt32Op{ + op := &projPowInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projPowInt16ConstInt64Op{ + op := &projPowInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt16ConstDecimalOp{ + op := &projPowInt16ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -30848,30 +29832,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt32ConstInt16Op{ + op := &projPowInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projPowInt32ConstInt32Op{ + op := &projPowInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPowInt32ConstInt64Op{ + op := &projPowInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt32ConstDecimalOp{ + op := &projPowInt32ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -30880,30 +29868,34 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt64ConstInt16Op{ + op := &projPowInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projPowInt64ConstInt32Op{ + op := &projPowInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projPowInt64ConstInt64Op{ + op := &projPowInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt64ConstDecimalOp{ + op := &projPowInt64ConstDecimalOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -30916,10 +29908,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projPowFloat64ConstFloat64Op{ + op := &projPowFloat64ConstFloat64Op{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -30935,10 +29928,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projConcatBytesConstBytesOp{ + op := &projConcatBytesConstBytesOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + return op, nil } } } @@ -30951,10 +29945,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projConcatJSONConstJSONOp{ + op := &projConcatJSONConstJSONOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -30967,10 +29962,12 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projConcatDatumConstDatumOp{ + op := &projConcatDatumConstDatumOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -30984,21 +29981,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt16ConstInt16Op{ + op := &projLShiftInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projLShiftInt16ConstInt32Op{ + op := &projLShiftInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt16ConstInt64Op{ + op := &projLShiftInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -31006,21 +30006,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt32ConstInt16Op{ + op := &projLShiftInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projLShiftInt32ConstInt32Op{ + op := &projLShiftInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt32ConstInt64Op{ + op := &projLShiftInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -31029,21 +30032,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt64ConstInt16Op{ + op := &projLShiftInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projLShiftInt64ConstInt32Op{ + op := &projLShiftInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt64ConstInt64Op{ + op := &projLShiftInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -31055,21 +30061,27 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftDatumConstInt16Op{ + op := &projLShiftDatumConstInt16Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projLShiftDatumConstInt32Op{ + op := &projLShiftDatumConstInt32Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projLShiftDatumConstInt64Op{ + op := &projLShiftDatumConstInt64Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -31083,21 +30095,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt16ConstInt16Op{ + op := &projRShiftInt16ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projRShiftInt16ConstInt32Op{ + op := &projRShiftInt16ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt16ConstInt64Op{ + op := &projRShiftInt16ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil } } case 32: @@ -31105,21 +30120,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt32ConstInt16Op{ + op := &projRShiftInt32ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case 32: - return &projRShiftInt32ConstInt32Op{ + op := &projRShiftInt32ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt32ConstInt64Op{ + op := &projRShiftInt32ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil } } case -1: @@ -31128,21 +30146,24 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt64ConstInt16Op{ + op := &projRShiftInt64ConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case 32: - return &projRShiftInt64ConstInt32Op{ + op := &projRShiftInt64ConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt64ConstInt64Op{ + op := &projRShiftInt64ConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -31154,21 +30175,27 @@ func GetProjectionLConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftDatumConstInt16Op{ + op := &projRShiftDatumConstInt16Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projRShiftDatumConstInt32Op{ + op := &projRShiftDatumConstInt32Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projRShiftDatumConstInt64Op{ + op := &projRShiftDatumConstInt64Op{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -31184,29 +30211,33 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValJSONConstBytesOp{ + op := &projJSONFetchValJSONConstBytesOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchValJSONConstInt16Op{ + op := &projJSONFetchValJSONConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case 32: - return &projJSONFetchValJSONConstInt32Op{ + op := &projJSONFetchValJSONConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case -1: default: - return &projJSONFetchValJSONConstInt64Op{ + op := &projJSONFetchValJSONConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -31222,29 +30253,33 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextJSONConstBytesOp{ + op := &projJSONFetchTextJSONConstBytesOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchTextJSONConstInt16Op{ + op := &projJSONFetchTextJSONConstInt16Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case 32: - return &projJSONFetchTextJSONConstInt32Op{ + op := &projJSONFetchTextJSONConstInt32Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil case -1: default: - return &projJSONFetchTextJSONConstInt64Op{ + op := &projJSONFetchTextJSONConstInt64Op{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -31260,10 +30295,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValPathJSONConstDatumOp{ + op := &projJSONFetchValPathJSONConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -31279,10 +30315,11 @@ func GetProjectionLConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextPathJSONConstDatumOp{ + op := &projJSONFetchTextPathJSONConstDatumOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } diff --git a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go index 3b2edae486a2..5def139fd856 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go @@ -83,6 +83,9 @@ func _ASSIGN(_, _, _, _, _, _ interface{}) { type _OP_CONST_NAME struct { projConstOpBase + // {{if .NeedsBinaryOverloadHelper}} + execgen.BinaryOverloadHelper + // {{end}} // {{if _IS_CONST_LEFT}} constArg _L_GO_TYPE // {{else}} @@ -91,12 +94,12 @@ type _OP_CONST_NAME struct { } func (p _OP_CONST_NAME) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // {{if .NeedsBinaryOverloadHelper}} + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper + // {{end}} batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -269,7 +272,6 @@ func GetProjection_CONST_SIDEConstOperator( allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, - overloadHelper: execgen.OverloadHelper{BinFn: binFn, EvalCtx: evalCtx}, } c := colconv.GetDatumToPhysicalFn(constType)(constArg) // {{if _IS_CONST_LEFT}} @@ -296,7 +298,7 @@ func GetProjection_CONST_SIDEConstOperator( switch rightType.Width() { // {{range .RightWidths}} case _RIGHT_TYPE_WIDTH: - return &_OP_CONST_NAME{ + op := &_OP_CONST_NAME{ projConstOpBase: projConstOpBase, // {{if _IS_CONST_LEFT}} // {{if eq $leftFamilyStr "typeconv.DatumVecCanonicalTypeFamily"}} @@ -311,7 +313,11 @@ func GetProjection_CONST_SIDEConstOperator( constArg: c.(_R_GO_TYPE), // {{end}} // {{end}} - }, nil + } + // {{if .NeedsBinaryOverloadHelper}} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + // {{end}} + return op, nil // {{end}} } // {{end}} @@ -338,7 +344,6 @@ func GetProjection_CONST_SIDEConstOperator( case tree._NAME: switch typeconv.TypeFamilyToCanonicalTypeFamily(leftType.Family()) { // {{range .LeftFamilies}} - // {{$leftFamilyStr := .LeftCanonicalFamilyStr}} case _LEFT_CANONICAL_TYPE_FAMILY: switch leftType.Width() { // {{range .LeftWidths}} diff --git a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go index 91a99fd8b352..c8d9fc423b2f 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go @@ -52,12 +52,6 @@ type projBitandInt16Int16ConstOp struct { } func (p projBitandInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -154,12 +148,6 @@ type projBitandInt16Int32ConstOp struct { } func (p projBitandInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -256,12 +244,6 @@ type projBitandInt16Int64ConstOp struct { } func (p projBitandInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -358,12 +340,6 @@ type projBitandInt32Int16ConstOp struct { } func (p projBitandInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -460,12 +436,6 @@ type projBitandInt32Int32ConstOp struct { } func (p projBitandInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -562,12 +532,6 @@ type projBitandInt32Int64ConstOp struct { } func (p projBitandInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -664,12 +628,6 @@ type projBitandInt64Int16ConstOp struct { } func (p projBitandInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -766,12 +724,6 @@ type projBitandInt64Int32ConstOp struct { } func (p projBitandInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -868,12 +820,6 @@ type projBitandInt64Int64ConstOp struct { } func (p projBitandInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -966,16 +912,15 @@ func (p projBitandInt64Int64ConstOp) Next() coldata.Batch { type projBitandDatumDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitandDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1098,12 +1043,6 @@ type projBitorInt16Int16ConstOp struct { } func (p projBitorInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1200,12 +1139,6 @@ type projBitorInt16Int32ConstOp struct { } func (p projBitorInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1302,12 +1235,6 @@ type projBitorInt16Int64ConstOp struct { } func (p projBitorInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1404,12 +1331,6 @@ type projBitorInt32Int16ConstOp struct { } func (p projBitorInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1506,12 +1427,6 @@ type projBitorInt32Int32ConstOp struct { } func (p projBitorInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1608,12 +1523,6 @@ type projBitorInt32Int64ConstOp struct { } func (p projBitorInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1710,12 +1619,6 @@ type projBitorInt64Int16ConstOp struct { } func (p projBitorInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1812,12 +1715,6 @@ type projBitorInt64Int32ConstOp struct { } func (p projBitorInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1914,12 +1811,6 @@ type projBitorInt64Int64ConstOp struct { } func (p projBitorInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2012,16 +1903,15 @@ func (p projBitorInt64Int64ConstOp) Next() coldata.Batch { type projBitorDatumDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitorDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2144,12 +2034,6 @@ type projBitxorInt16Int16ConstOp struct { } func (p projBitxorInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2246,12 +2130,6 @@ type projBitxorInt16Int32ConstOp struct { } func (p projBitxorInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2348,12 +2226,6 @@ type projBitxorInt16Int64ConstOp struct { } func (p projBitxorInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2450,12 +2322,6 @@ type projBitxorInt32Int16ConstOp struct { } func (p projBitxorInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2552,12 +2418,6 @@ type projBitxorInt32Int32ConstOp struct { } func (p projBitxorInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2654,12 +2514,6 @@ type projBitxorInt32Int64ConstOp struct { } func (p projBitxorInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2756,12 +2610,6 @@ type projBitxorInt64Int16ConstOp struct { } func (p projBitxorInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2858,12 +2706,6 @@ type projBitxorInt64Int32ConstOp struct { } func (p projBitxorInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2960,12 +2802,6 @@ type projBitxorInt64Int64ConstOp struct { } func (p projBitxorInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3058,16 +2894,15 @@ func (p projBitxorInt64Int64ConstOp) Next() coldata.Batch { type projBitxorDatumDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projBitxorDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3190,12 +3025,6 @@ type projPlusDecimalInt16ConstOp struct { } func (p projPlusDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3320,12 +3149,6 @@ type projPlusDecimalInt32ConstOp struct { } func (p projPlusDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3450,12 +3273,6 @@ type projPlusDecimalInt64ConstOp struct { } func (p projPlusDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3580,12 +3397,6 @@ type projPlusDecimalDecimalConstOp struct { } func (p projPlusDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3706,12 +3517,6 @@ type projPlusInt16Int16ConstOp struct { } func (p projPlusInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3832,12 +3637,6 @@ type projPlusInt16Int32ConstOp struct { } func (p projPlusInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3958,12 +3757,6 @@ type projPlusInt16Int64ConstOp struct { } func (p projPlusInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4084,12 +3877,6 @@ type projPlusInt16DecimalConstOp struct { } func (p projPlusInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4214,16 +4001,15 @@ func (p projPlusInt16DecimalConstOp) Next() coldata.Batch { type projPlusInt16DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusInt16DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4362,12 +4148,6 @@ type projPlusInt32Int16ConstOp struct { } func (p projPlusInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4488,12 +4268,6 @@ type projPlusInt32Int32ConstOp struct { } func (p projPlusInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4614,12 +4388,6 @@ type projPlusInt32Int64ConstOp struct { } func (p projPlusInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4740,12 +4508,6 @@ type projPlusInt32DecimalConstOp struct { } func (p projPlusInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4870,16 +4632,15 @@ func (p projPlusInt32DecimalConstOp) Next() coldata.Batch { type projPlusInt32DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusInt32DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5018,12 +4779,6 @@ type projPlusInt64Int16ConstOp struct { } func (p projPlusInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5144,12 +4899,6 @@ type projPlusInt64Int32ConstOp struct { } func (p projPlusInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5270,12 +5019,6 @@ type projPlusInt64Int64ConstOp struct { } func (p projPlusInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5396,12 +5139,6 @@ type projPlusInt64DecimalConstOp struct { } func (p projPlusInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5526,16 +5263,15 @@ func (p projPlusInt64DecimalConstOp) Next() coldata.Batch { type projPlusInt64DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusInt64DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5674,12 +5410,6 @@ type projPlusFloat64Float64ConstOp struct { } func (p projPlusFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5788,12 +5518,6 @@ type projPlusTimestampIntervalConstOp struct { } func (p projPlusTimestampIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5902,12 +5626,6 @@ type projPlusIntervalTimestampConstOp struct { } func (p projPlusIntervalTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6016,12 +5734,6 @@ type projPlusIntervalIntervalConstOp struct { } func (p projPlusIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6106,16 +5818,15 @@ func (p projPlusIntervalIntervalConstOp) Next() coldata.Batch { type projPlusIntervalDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projPlusIntervalDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6250,16 +5961,15 @@ func (p projPlusIntervalDatumConstOp) Next() coldata.Batch { type projPlusDatumIntervalConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg duration.Duration } func (p projPlusDatumIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6396,16 +6106,15 @@ func (p projPlusDatumIntervalConstOp) Next() coldata.Batch { type projPlusDatumInt16ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projPlusDatumInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6542,16 +6251,15 @@ func (p projPlusDatumInt16ConstOp) Next() coldata.Batch { type projPlusDatumInt32ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projPlusDatumInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6688,16 +6396,15 @@ func (p projPlusDatumInt32ConstOp) Next() coldata.Batch { type projPlusDatumInt64ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projPlusDatumInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6838,12 +6545,6 @@ type projMinusDecimalInt16ConstOp struct { } func (p projMinusDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6968,12 +6669,6 @@ type projMinusDecimalInt32ConstOp struct { } func (p projMinusDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7098,12 +6793,6 @@ type projMinusDecimalInt64ConstOp struct { } func (p projMinusDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7228,12 +6917,6 @@ type projMinusDecimalDecimalConstOp struct { } func (p projMinusDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7354,12 +7037,6 @@ type projMinusInt16Int16ConstOp struct { } func (p projMinusInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7480,12 +7157,6 @@ type projMinusInt16Int32ConstOp struct { } func (p projMinusInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7606,12 +7277,6 @@ type projMinusInt16Int64ConstOp struct { } func (p projMinusInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7732,12 +7397,6 @@ type projMinusInt16DecimalConstOp struct { } func (p projMinusInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7862,16 +7521,15 @@ func (p projMinusInt16DecimalConstOp) Next() coldata.Batch { type projMinusInt16DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusInt16DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8010,12 +7668,6 @@ type projMinusInt32Int16ConstOp struct { } func (p projMinusInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8136,12 +7788,6 @@ type projMinusInt32Int32ConstOp struct { } func (p projMinusInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8262,12 +7908,6 @@ type projMinusInt32Int64ConstOp struct { } func (p projMinusInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8388,12 +8028,6 @@ type projMinusInt32DecimalConstOp struct { } func (p projMinusInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8518,16 +8152,15 @@ func (p projMinusInt32DecimalConstOp) Next() coldata.Batch { type projMinusInt32DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusInt32DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8666,12 +8299,6 @@ type projMinusInt64Int16ConstOp struct { } func (p projMinusInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8792,12 +8419,6 @@ type projMinusInt64Int32ConstOp struct { } func (p projMinusInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8918,12 +8539,6 @@ type projMinusInt64Int64ConstOp struct { } func (p projMinusInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9044,12 +8659,6 @@ type projMinusInt64DecimalConstOp struct { } func (p projMinusInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9174,16 +8783,15 @@ func (p projMinusInt64DecimalConstOp) Next() coldata.Batch { type projMinusInt64DatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusInt64DatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9322,12 +8930,6 @@ type projMinusFloat64Float64ConstOp struct { } func (p projMinusFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9436,12 +9038,6 @@ type projMinusTimestampTimestampConstOp struct { } func (p projMinusTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9542,12 +9138,6 @@ type projMinusTimestampIntervalConstOp struct { } func (p projMinusTimestampIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9656,12 +9246,6 @@ type projMinusIntervalIntervalConstOp struct { } func (p projMinusIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9746,16 +9330,15 @@ func (p projMinusIntervalIntervalConstOp) Next() coldata.Batch { type projMinusIntervalDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusIntervalDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9894,12 +9477,6 @@ type projMinusJSONBytesConstOp struct { } func (p projMinusJSONBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10022,12 +9599,6 @@ type projMinusJSONInt16ConstOp struct { } func (p projMinusJSONInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10136,12 +9707,6 @@ type projMinusJSONInt32ConstOp struct { } func (p projMinusJSONInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10250,12 +9815,6 @@ type projMinusJSONInt64ConstOp struct { } func (p projMinusJSONInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10360,16 +9919,15 @@ func (p projMinusJSONInt64ConstOp) Next() coldata.Batch { type projMinusDatumDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projMinusDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10488,16 +10046,15 @@ func (p projMinusDatumDatumConstOp) Next() coldata.Batch { type projMinusDatumIntervalConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg duration.Duration } func (p projMinusDatumIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10634,16 +10191,15 @@ func (p projMinusDatumIntervalConstOp) Next() coldata.Batch { type projMinusDatumBytesConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg []byte } func (p projMinusDatumBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10778,16 +10334,15 @@ func (p projMinusDatumBytesConstOp) Next() coldata.Batch { type projMinusDatumInt16ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projMinusDatumInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10924,16 +10479,15 @@ func (p projMinusDatumInt16ConstOp) Next() coldata.Batch { type projMinusDatumInt32ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projMinusDatumInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11070,16 +10624,15 @@ func (p projMinusDatumInt32ConstOp) Next() coldata.Batch { type projMinusDatumInt64ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projMinusDatumInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11220,12 +10773,6 @@ type projMultDecimalInt16ConstOp struct { } func (p projMultDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11350,12 +10897,6 @@ type projMultDecimalInt32ConstOp struct { } func (p projMultDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11480,12 +11021,6 @@ type projMultDecimalInt64ConstOp struct { } func (p projMultDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11610,12 +11145,6 @@ type projMultDecimalDecimalConstOp struct { } func (p projMultDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11736,12 +11265,6 @@ type projMultDecimalIntervalConstOp struct { } func (p projMultDecimalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11850,12 +11373,6 @@ type projMultInt16Int16ConstOp struct { } func (p projMultInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12008,12 +11525,6 @@ type projMultInt16Int32ConstOp struct { } func (p projMultInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12166,12 +11677,6 @@ type projMultInt16Int64ConstOp struct { } func (p projMultInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12324,12 +11829,6 @@ type projMultInt16DecimalConstOp struct { } func (p projMultInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12458,12 +11957,6 @@ type projMultInt16IntervalConstOp struct { } func (p projMultInt16IntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12552,12 +12045,6 @@ type projMultInt32Int16ConstOp struct { } func (p projMultInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12710,12 +12197,6 @@ type projMultInt32Int32ConstOp struct { } func (p projMultInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12868,12 +12349,6 @@ type projMultInt32Int64ConstOp struct { } func (p projMultInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13026,12 +12501,6 @@ type projMultInt32DecimalConstOp struct { } func (p projMultInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13160,12 +12629,6 @@ type projMultInt32IntervalConstOp struct { } func (p projMultInt32IntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13254,12 +12717,6 @@ type projMultInt64Int16ConstOp struct { } func (p projMultInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13412,12 +12869,6 @@ type projMultInt64Int32ConstOp struct { } func (p projMultInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13570,12 +13021,6 @@ type projMultInt64Int64ConstOp struct { } func (p projMultInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13728,12 +13173,6 @@ type projMultInt64DecimalConstOp struct { } func (p projMultInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13862,12 +13301,6 @@ type projMultInt64IntervalConstOp struct { } func (p projMultInt64IntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13956,12 +13389,6 @@ type projMultFloat64Float64ConstOp struct { } func (p projMultFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14070,12 +13497,6 @@ type projMultFloat64IntervalConstOp struct { } func (p projMultFloat64IntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14164,12 +13585,6 @@ type projMultIntervalInt16ConstOp struct { } func (p projMultIntervalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14258,12 +13673,6 @@ type projMultIntervalInt32ConstOp struct { } func (p projMultIntervalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14352,12 +13761,6 @@ type projMultIntervalInt64ConstOp struct { } func (p projMultIntervalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14446,12 +13849,6 @@ type projMultIntervalFloat64ConstOp struct { } func (p projMultIntervalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14540,12 +13937,6 @@ type projMultIntervalDecimalConstOp struct { } func (p projMultIntervalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14654,12 +14045,6 @@ type projDivDecimalInt16ConstOp struct { } func (p projDivDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14800,12 +14185,6 @@ type projDivDecimalInt32ConstOp struct { } func (p projDivDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14946,12 +14325,6 @@ type projDivDecimalInt64ConstOp struct { } func (p projDivDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15092,12 +14465,6 @@ type projDivDecimalDecimalConstOp struct { } func (p projDivDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15234,12 +14601,6 @@ type projDivInt16Int16ConstOp struct { } func (p projDivInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15376,12 +14737,6 @@ type projDivInt16Int32ConstOp struct { } func (p projDivInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15518,12 +14873,6 @@ type projDivInt16Int64ConstOp struct { } func (p projDivInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15660,12 +15009,6 @@ type projDivInt16DecimalConstOp struct { } func (p projDivInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15810,12 +15153,6 @@ type projDivInt32Int16ConstOp struct { } func (p projDivInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15952,12 +15289,6 @@ type projDivInt32Int32ConstOp struct { } func (p projDivInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16094,12 +15425,6 @@ type projDivInt32Int64ConstOp struct { } func (p projDivInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16236,12 +15561,6 @@ type projDivInt32DecimalConstOp struct { } func (p projDivInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16386,12 +15705,6 @@ type projDivInt64Int16ConstOp struct { } func (p projDivInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16528,12 +15841,6 @@ type projDivInt64Int32ConstOp struct { } func (p projDivInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16670,12 +15977,6 @@ type projDivInt64Int64ConstOp struct { } func (p projDivInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16812,12 +16113,6 @@ type projDivInt64DecimalConstOp struct { } func (p projDivInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16962,12 +16257,6 @@ type projDivFloat64Float64ConstOp struct { } func (p projDivFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17092,12 +16381,6 @@ type projDivIntervalInt64ConstOp struct { } func (p projDivIntervalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17202,12 +16485,6 @@ type projDivIntervalFloat64ConstOp struct { } func (p projDivIntervalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17312,12 +16589,6 @@ type projFloorDivDecimalInt16ConstOp struct { } func (p projFloorDivDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17458,12 +16729,6 @@ type projFloorDivDecimalInt32ConstOp struct { } func (p projFloorDivDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17604,12 +16869,6 @@ type projFloorDivDecimalInt64ConstOp struct { } func (p projFloorDivDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17750,12 +17009,6 @@ type projFloorDivDecimalDecimalConstOp struct { } func (p projFloorDivDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17892,12 +17145,6 @@ type projFloorDivInt16Int16ConstOp struct { } func (p projFloorDivInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18014,12 +17261,6 @@ type projFloorDivInt16Int32ConstOp struct { } func (p projFloorDivInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18136,12 +17377,6 @@ type projFloorDivInt16Int64ConstOp struct { } func (p projFloorDivInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18258,12 +17493,6 @@ type projFloorDivInt16DecimalConstOp struct { } func (p projFloorDivInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18408,12 +17637,6 @@ type projFloorDivInt32Int16ConstOp struct { } func (p projFloorDivInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18530,12 +17753,6 @@ type projFloorDivInt32Int32ConstOp struct { } func (p projFloorDivInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18652,12 +17869,6 @@ type projFloorDivInt32Int64ConstOp struct { } func (p projFloorDivInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18774,12 +17985,6 @@ type projFloorDivInt32DecimalConstOp struct { } func (p projFloorDivInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18924,12 +18129,6 @@ type projFloorDivInt64Int16ConstOp struct { } func (p projFloorDivInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19046,12 +18245,6 @@ type projFloorDivInt64Int32ConstOp struct { } func (p projFloorDivInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19168,12 +18361,6 @@ type projFloorDivInt64Int64ConstOp struct { } func (p projFloorDivInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19290,12 +18477,6 @@ type projFloorDivInt64DecimalConstOp struct { } func (p projFloorDivInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19440,12 +18621,6 @@ type projFloorDivFloat64Float64ConstOp struct { } func (p projFloorDivFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19570,12 +18745,6 @@ type projModDecimalInt16ConstOp struct { } func (p projModDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19716,12 +18885,6 @@ type projModDecimalInt32ConstOp struct { } func (p projModDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19862,12 +19025,6 @@ type projModDecimalInt64ConstOp struct { } func (p projModDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20008,12 +19165,6 @@ type projModDecimalDecimalConstOp struct { } func (p projModDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20150,12 +19301,6 @@ type projModInt16Int16ConstOp struct { } func (p projModInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20272,12 +19417,6 @@ type projModInt16Int32ConstOp struct { } func (p projModInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20394,12 +19533,6 @@ type projModInt16Int64ConstOp struct { } func (p projModInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20516,12 +19649,6 @@ type projModInt16DecimalConstOp struct { } func (p projModInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20666,12 +19793,6 @@ type projModInt32Int16ConstOp struct { } func (p projModInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20788,12 +19909,6 @@ type projModInt32Int32ConstOp struct { } func (p projModInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20910,12 +20025,6 @@ type projModInt32Int64ConstOp struct { } func (p projModInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21032,12 +20141,6 @@ type projModInt32DecimalConstOp struct { } func (p projModInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21182,12 +20285,6 @@ type projModInt64Int16ConstOp struct { } func (p projModInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21304,12 +20401,6 @@ type projModInt64Int32ConstOp struct { } func (p projModInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21426,12 +20517,6 @@ type projModInt64Int64ConstOp struct { } func (p projModInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21548,12 +20633,6 @@ type projModInt64DecimalConstOp struct { } func (p projModInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21698,12 +20777,6 @@ type projModFloat64Float64ConstOp struct { } func (p projModFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21828,12 +20901,6 @@ type projPowDecimalInt16ConstOp struct { } func (p projPowDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21958,12 +21025,6 @@ type projPowDecimalInt32ConstOp struct { } func (p projPowDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22088,12 +21149,6 @@ type projPowDecimalInt64ConstOp struct { } func (p projPowDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22218,12 +21273,6 @@ type projPowDecimalDecimalConstOp struct { } func (p projPowDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22344,12 +21393,6 @@ type projPowInt16Int16ConstOp struct { } func (p projPowInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22494,12 +21537,6 @@ type projPowInt16Int32ConstOp struct { } func (p projPowInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22644,12 +21681,6 @@ type projPowInt16Int64ConstOp struct { } func (p projPowInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22794,12 +21825,6 @@ type projPowInt16DecimalConstOp struct { } func (p projPowInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22928,12 +21953,6 @@ type projPowInt32Int16ConstOp struct { } func (p projPowInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23078,12 +22097,6 @@ type projPowInt32Int32ConstOp struct { } func (p projPowInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23228,12 +22241,6 @@ type projPowInt32Int64ConstOp struct { } func (p projPowInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23378,12 +22385,6 @@ type projPowInt32DecimalConstOp struct { } func (p projPowInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23512,12 +22513,6 @@ type projPowInt64Int16ConstOp struct { } func (p projPowInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23662,12 +22657,6 @@ type projPowInt64Int32ConstOp struct { } func (p projPowInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23812,12 +22801,6 @@ type projPowInt64Int64ConstOp struct { } func (p projPowInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23962,12 +22945,6 @@ type projPowInt64DecimalConstOp struct { } func (p projPowInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24096,12 +23073,6 @@ type projPowFloat64Float64ConstOp struct { } func (p projPowFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24210,12 +23181,6 @@ type projConcatBytesBytesConstOp struct { } func (p projConcatBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24330,12 +23295,6 @@ type projConcatJSONJSONConstOp struct { } func (p projConcatJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24442,16 +23401,15 @@ func (p projConcatJSONJSONConstOp) Next() coldata.Batch { type projConcatDatumDatumConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg interface{} } func (p projConcatDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24574,12 +23532,6 @@ type projLShiftInt16Int16ConstOp struct { } func (p projLShiftInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24700,12 +23652,6 @@ type projLShiftInt16Int32ConstOp struct { } func (p projLShiftInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24826,12 +23772,6 @@ type projLShiftInt16Int64ConstOp struct { } func (p projLShiftInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24952,12 +23892,6 @@ type projLShiftInt32Int16ConstOp struct { } func (p projLShiftInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25078,12 +24012,6 @@ type projLShiftInt32Int32ConstOp struct { } func (p projLShiftInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25204,12 +24132,6 @@ type projLShiftInt32Int64ConstOp struct { } func (p projLShiftInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25330,12 +24252,6 @@ type projLShiftInt64Int16ConstOp struct { } func (p projLShiftInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25456,12 +24372,6 @@ type projLShiftInt64Int32ConstOp struct { } func (p projLShiftInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25582,12 +24492,6 @@ type projLShiftInt64Int64ConstOp struct { } func (p projLShiftInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25704,16 +24608,15 @@ func (p projLShiftInt64Int64ConstOp) Next() coldata.Batch { type projLShiftDatumInt16ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projLShiftDatumInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25850,16 +24753,15 @@ func (p projLShiftDatumInt16ConstOp) Next() coldata.Batch { type projLShiftDatumInt32ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projLShiftDatumInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25996,16 +24898,15 @@ func (p projLShiftDatumInt32ConstOp) Next() coldata.Batch { type projLShiftDatumInt64ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projLShiftDatumInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26146,12 +25047,6 @@ type projRShiftInt16Int16ConstOp struct { } func (p projRShiftInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26272,12 +25167,6 @@ type projRShiftInt16Int32ConstOp struct { } func (p projRShiftInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26398,12 +25287,6 @@ type projRShiftInt16Int64ConstOp struct { } func (p projRShiftInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26524,12 +25407,6 @@ type projRShiftInt32Int16ConstOp struct { } func (p projRShiftInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26650,12 +25527,6 @@ type projRShiftInt32Int32ConstOp struct { } func (p projRShiftInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26776,12 +25647,6 @@ type projRShiftInt32Int64ConstOp struct { } func (p projRShiftInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26902,12 +25767,6 @@ type projRShiftInt64Int16ConstOp struct { } func (p projRShiftInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27028,12 +25887,6 @@ type projRShiftInt64Int32ConstOp struct { } func (p projRShiftInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27154,12 +26007,6 @@ type projRShiftInt64Int64ConstOp struct { } func (p projRShiftInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27276,16 +26123,15 @@ func (p projRShiftInt64Int64ConstOp) Next() coldata.Batch { type projRShiftDatumInt16ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int16 } func (p projRShiftDatumInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27422,16 +26268,15 @@ func (p projRShiftDatumInt16ConstOp) Next() coldata.Batch { type projRShiftDatumInt32ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int32 } func (p projRShiftDatumInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27568,16 +26413,15 @@ func (p projRShiftDatumInt32ConstOp) Next() coldata.Batch { type projRShiftDatumInt64ConstOp struct { projConstOpBase + execgen.BinaryOverloadHelper constArg int64 } func (p projRShiftDatumInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27718,12 +26562,6 @@ type projJSONFetchValJSONBytesConstOp struct { } func (p projJSONFetchValJSONBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27858,12 +26696,6 @@ type projJSONFetchValJSONInt16ConstOp struct { } func (p projJSONFetchValJSONInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27988,12 +26820,6 @@ type projJSONFetchValJSONInt32ConstOp struct { } func (p projJSONFetchValJSONInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28118,12 +26944,6 @@ type projJSONFetchValJSONInt64ConstOp struct { } func (p projJSONFetchValJSONInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28248,12 +27068,6 @@ type projJSONFetchTextJSONBytesConstOp struct { } func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28424,12 +27238,6 @@ type projJSONFetchTextJSONInt16ConstOp struct { } func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28590,12 +27398,6 @@ type projJSONFetchTextJSONInt32ConstOp struct { } func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28756,12 +27558,6 @@ type projJSONFetchTextJSONInt64ConstOp struct { } func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28922,12 +27718,6 @@ type projJSONFetchValPathJSONDatumConstOp struct { } func (p projJSONFetchValPathJSONDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29050,12 +27840,6 @@ type projJSONFetchTextPathJSONDatumConstOp struct { } func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29218,12 +28002,6 @@ type projEQBoolBoolConstOp struct { } func (p projEQBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29368,12 +28146,6 @@ type projEQBytesBytesConstOp struct { } func (p projEQBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29484,12 +28256,6 @@ type projEQDecimalInt16ConstOp struct { } func (p projEQDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29626,12 +28392,6 @@ type projEQDecimalInt32ConstOp struct { } func (p projEQDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29768,12 +28528,6 @@ type projEQDecimalInt64ConstOp struct { } func (p projEQDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29910,12 +28664,6 @@ type projEQDecimalFloat64ConstOp struct { } func (p projEQDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30060,12 +28808,6 @@ type projEQDecimalDecimalConstOp struct { } func (p projEQDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30178,12 +28920,6 @@ type projEQInt16Int16ConstOp struct { } func (p projEQInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30340,12 +29076,6 @@ type projEQInt16Int32ConstOp struct { } func (p projEQInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30502,12 +29232,6 @@ type projEQInt16Int64ConstOp struct { } func (p projEQInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30664,12 +29388,6 @@ type projEQInt16Float64ConstOp struct { } func (p projEQInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30858,12 +29576,6 @@ type projEQInt16DecimalConstOp struct { } func (p projEQInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31000,12 +29712,6 @@ type projEQInt32Int16ConstOp struct { } func (p projEQInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31162,12 +29868,6 @@ type projEQInt32Int32ConstOp struct { } func (p projEQInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31324,12 +30024,6 @@ type projEQInt32Int64ConstOp struct { } func (p projEQInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31486,12 +30180,6 @@ type projEQInt32Float64ConstOp struct { } func (p projEQInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31680,12 +30368,6 @@ type projEQInt32DecimalConstOp struct { } func (p projEQInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31822,12 +30504,6 @@ type projEQInt64Int16ConstOp struct { } func (p projEQInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31984,12 +30660,6 @@ type projEQInt64Int32ConstOp struct { } func (p projEQInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32146,12 +30816,6 @@ type projEQInt64Int64ConstOp struct { } func (p projEQInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32308,12 +30972,6 @@ type projEQInt64Float64ConstOp struct { } func (p projEQInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32502,12 +31160,6 @@ type projEQInt64DecimalConstOp struct { } func (p projEQInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32644,12 +31296,6 @@ type projEQFloat64Int16ConstOp struct { } func (p projEQFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32838,12 +31484,6 @@ type projEQFloat64Int32ConstOp struct { } func (p projEQFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33032,12 +31672,6 @@ type projEQFloat64Int64ConstOp struct { } func (p projEQFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33226,12 +31860,6 @@ type projEQFloat64Float64ConstOp struct { } func (p projEQFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33420,12 +32048,6 @@ type projEQFloat64DecimalConstOp struct { } func (p projEQFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33570,12 +32192,6 @@ type projEQTimestampTimestampConstOp struct { } func (p projEQTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33716,12 +32332,6 @@ type projEQIntervalIntervalConstOp struct { } func (p projEQIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33834,12 +32444,6 @@ type projEQJSONJSONConstOp struct { } func (p projEQJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33974,12 +32578,6 @@ type projEQDatumDatumConstOp struct { } func (p projEQDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34098,12 +32696,6 @@ type projNEBoolBoolConstOp struct { } func (p projNEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34248,12 +32840,6 @@ type projNEBytesBytesConstOp struct { } func (p projNEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34364,12 +32950,6 @@ type projNEDecimalInt16ConstOp struct { } func (p projNEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34506,12 +33086,6 @@ type projNEDecimalInt32ConstOp struct { } func (p projNEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34648,12 +33222,6 @@ type projNEDecimalInt64ConstOp struct { } func (p projNEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34790,12 +33358,6 @@ type projNEDecimalFloat64ConstOp struct { } func (p projNEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34940,12 +33502,6 @@ type projNEDecimalDecimalConstOp struct { } func (p projNEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35058,12 +33614,6 @@ type projNEInt16Int16ConstOp struct { } func (p projNEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35220,12 +33770,6 @@ type projNEInt16Int32ConstOp struct { } func (p projNEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35382,12 +33926,6 @@ type projNEInt16Int64ConstOp struct { } func (p projNEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35544,12 +34082,6 @@ type projNEInt16Float64ConstOp struct { } func (p projNEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35738,12 +34270,6 @@ type projNEInt16DecimalConstOp struct { } func (p projNEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35880,12 +34406,6 @@ type projNEInt32Int16ConstOp struct { } func (p projNEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36042,12 +34562,6 @@ type projNEInt32Int32ConstOp struct { } func (p projNEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36204,12 +34718,6 @@ type projNEInt32Int64ConstOp struct { } func (p projNEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36366,12 +34874,6 @@ type projNEInt32Float64ConstOp struct { } func (p projNEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36560,12 +35062,6 @@ type projNEInt32DecimalConstOp struct { } func (p projNEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36702,12 +35198,6 @@ type projNEInt64Int16ConstOp struct { } func (p projNEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36864,12 +35354,6 @@ type projNEInt64Int32ConstOp struct { } func (p projNEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37026,12 +35510,6 @@ type projNEInt64Int64ConstOp struct { } func (p projNEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37188,12 +35666,6 @@ type projNEInt64Float64ConstOp struct { } func (p projNEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37382,12 +35854,6 @@ type projNEInt64DecimalConstOp struct { } func (p projNEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37524,12 +35990,6 @@ type projNEFloat64Int16ConstOp struct { } func (p projNEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37718,12 +36178,6 @@ type projNEFloat64Int32ConstOp struct { } func (p projNEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37912,12 +36366,6 @@ type projNEFloat64Int64ConstOp struct { } func (p projNEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38106,12 +36554,6 @@ type projNEFloat64Float64ConstOp struct { } func (p projNEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38300,12 +36742,6 @@ type projNEFloat64DecimalConstOp struct { } func (p projNEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38450,12 +36886,6 @@ type projNETimestampTimestampConstOp struct { } func (p projNETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38596,12 +37026,6 @@ type projNEIntervalIntervalConstOp struct { } func (p projNEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38714,12 +37138,6 @@ type projNEJSONJSONConstOp struct { } func (p projNEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38854,12 +37272,6 @@ type projNEDatumDatumConstOp struct { } func (p projNEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38978,12 +37390,6 @@ type projLTBoolBoolConstOp struct { } func (p projLTBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39128,12 +37534,6 @@ type projLTBytesBytesConstOp struct { } func (p projLTBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39244,12 +37644,6 @@ type projLTDecimalInt16ConstOp struct { } func (p projLTDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39386,12 +37780,6 @@ type projLTDecimalInt32ConstOp struct { } func (p projLTDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39528,12 +37916,6 @@ type projLTDecimalInt64ConstOp struct { } func (p projLTDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39670,12 +38052,6 @@ type projLTDecimalFloat64ConstOp struct { } func (p projLTDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39820,12 +38196,6 @@ type projLTDecimalDecimalConstOp struct { } func (p projLTDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39938,12 +38308,6 @@ type projLTInt16Int16ConstOp struct { } func (p projLTInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40100,12 +38464,6 @@ type projLTInt16Int32ConstOp struct { } func (p projLTInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40262,12 +38620,6 @@ type projLTInt16Int64ConstOp struct { } func (p projLTInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40424,12 +38776,6 @@ type projLTInt16Float64ConstOp struct { } func (p projLTInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40618,12 +38964,6 @@ type projLTInt16DecimalConstOp struct { } func (p projLTInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40760,12 +39100,6 @@ type projLTInt32Int16ConstOp struct { } func (p projLTInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40922,12 +39256,6 @@ type projLTInt32Int32ConstOp struct { } func (p projLTInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41084,12 +39412,6 @@ type projLTInt32Int64ConstOp struct { } func (p projLTInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41246,12 +39568,6 @@ type projLTInt32Float64ConstOp struct { } func (p projLTInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41440,12 +39756,6 @@ type projLTInt32DecimalConstOp struct { } func (p projLTInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41582,12 +39892,6 @@ type projLTInt64Int16ConstOp struct { } func (p projLTInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41744,12 +40048,6 @@ type projLTInt64Int32ConstOp struct { } func (p projLTInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41906,12 +40204,6 @@ type projLTInt64Int64ConstOp struct { } func (p projLTInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42068,12 +40360,6 @@ type projLTInt64Float64ConstOp struct { } func (p projLTInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42262,12 +40548,6 @@ type projLTInt64DecimalConstOp struct { } func (p projLTInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42404,12 +40684,6 @@ type projLTFloat64Int16ConstOp struct { } func (p projLTFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42598,12 +40872,6 @@ type projLTFloat64Int32ConstOp struct { } func (p projLTFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42792,12 +41060,6 @@ type projLTFloat64Int64ConstOp struct { } func (p projLTFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42986,12 +41248,6 @@ type projLTFloat64Float64ConstOp struct { } func (p projLTFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43180,12 +41436,6 @@ type projLTFloat64DecimalConstOp struct { } func (p projLTFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43330,12 +41580,6 @@ type projLTTimestampTimestampConstOp struct { } func (p projLTTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43476,12 +41720,6 @@ type projLTIntervalIntervalConstOp struct { } func (p projLTIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43594,12 +41832,6 @@ type projLTJSONJSONConstOp struct { } func (p projLTJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43734,12 +41966,6 @@ type projLTDatumDatumConstOp struct { } func (p projLTDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43858,12 +42084,6 @@ type projLEBoolBoolConstOp struct { } func (p projLEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44008,12 +42228,6 @@ type projLEBytesBytesConstOp struct { } func (p projLEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44124,12 +42338,6 @@ type projLEDecimalInt16ConstOp struct { } func (p projLEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44266,12 +42474,6 @@ type projLEDecimalInt32ConstOp struct { } func (p projLEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44408,12 +42610,6 @@ type projLEDecimalInt64ConstOp struct { } func (p projLEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44550,12 +42746,6 @@ type projLEDecimalFloat64ConstOp struct { } func (p projLEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44700,12 +42890,6 @@ type projLEDecimalDecimalConstOp struct { } func (p projLEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44818,12 +43002,6 @@ type projLEInt16Int16ConstOp struct { } func (p projLEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44980,12 +43158,6 @@ type projLEInt16Int32ConstOp struct { } func (p projLEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45142,12 +43314,6 @@ type projLEInt16Int64ConstOp struct { } func (p projLEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45304,12 +43470,6 @@ type projLEInt16Float64ConstOp struct { } func (p projLEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45498,12 +43658,6 @@ type projLEInt16DecimalConstOp struct { } func (p projLEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45640,12 +43794,6 @@ type projLEInt32Int16ConstOp struct { } func (p projLEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45802,12 +43950,6 @@ type projLEInt32Int32ConstOp struct { } func (p projLEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45964,12 +44106,6 @@ type projLEInt32Int64ConstOp struct { } func (p projLEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46126,12 +44262,6 @@ type projLEInt32Float64ConstOp struct { } func (p projLEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46320,12 +44450,6 @@ type projLEInt32DecimalConstOp struct { } func (p projLEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46462,12 +44586,6 @@ type projLEInt64Int16ConstOp struct { } func (p projLEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46624,12 +44742,6 @@ type projLEInt64Int32ConstOp struct { } func (p projLEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46786,12 +44898,6 @@ type projLEInt64Int64ConstOp struct { } func (p projLEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46948,12 +45054,6 @@ type projLEInt64Float64ConstOp struct { } func (p projLEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47142,12 +45242,6 @@ type projLEInt64DecimalConstOp struct { } func (p projLEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47284,12 +45378,6 @@ type projLEFloat64Int16ConstOp struct { } func (p projLEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47478,12 +45566,6 @@ type projLEFloat64Int32ConstOp struct { } func (p projLEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47672,12 +45754,6 @@ type projLEFloat64Int64ConstOp struct { } func (p projLEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47866,12 +45942,6 @@ type projLEFloat64Float64ConstOp struct { } func (p projLEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48060,12 +46130,6 @@ type projLEFloat64DecimalConstOp struct { } func (p projLEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48210,12 +46274,6 @@ type projLETimestampTimestampConstOp struct { } func (p projLETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48356,12 +46414,6 @@ type projLEIntervalIntervalConstOp struct { } func (p projLEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48474,12 +46526,6 @@ type projLEJSONJSONConstOp struct { } func (p projLEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48614,12 +46660,6 @@ type projLEDatumDatumConstOp struct { } func (p projLEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48738,12 +46778,6 @@ type projGTBoolBoolConstOp struct { } func (p projGTBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48888,12 +46922,6 @@ type projGTBytesBytesConstOp struct { } func (p projGTBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49004,12 +47032,6 @@ type projGTDecimalInt16ConstOp struct { } func (p projGTDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49146,12 +47168,6 @@ type projGTDecimalInt32ConstOp struct { } func (p projGTDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49288,12 +47304,6 @@ type projGTDecimalInt64ConstOp struct { } func (p projGTDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49430,12 +47440,6 @@ type projGTDecimalFloat64ConstOp struct { } func (p projGTDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49580,12 +47584,6 @@ type projGTDecimalDecimalConstOp struct { } func (p projGTDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49698,12 +47696,6 @@ type projGTInt16Int16ConstOp struct { } func (p projGTInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49860,12 +47852,6 @@ type projGTInt16Int32ConstOp struct { } func (p projGTInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50022,12 +48008,6 @@ type projGTInt16Int64ConstOp struct { } func (p projGTInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50184,12 +48164,6 @@ type projGTInt16Float64ConstOp struct { } func (p projGTInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50378,12 +48352,6 @@ type projGTInt16DecimalConstOp struct { } func (p projGTInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50520,12 +48488,6 @@ type projGTInt32Int16ConstOp struct { } func (p projGTInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50682,12 +48644,6 @@ type projGTInt32Int32ConstOp struct { } func (p projGTInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50844,12 +48800,6 @@ type projGTInt32Int64ConstOp struct { } func (p projGTInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51006,12 +48956,6 @@ type projGTInt32Float64ConstOp struct { } func (p projGTInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51200,12 +49144,6 @@ type projGTInt32DecimalConstOp struct { } func (p projGTInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51342,12 +49280,6 @@ type projGTInt64Int16ConstOp struct { } func (p projGTInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51504,12 +49436,6 @@ type projGTInt64Int32ConstOp struct { } func (p projGTInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51666,12 +49592,6 @@ type projGTInt64Int64ConstOp struct { } func (p projGTInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51828,12 +49748,6 @@ type projGTInt64Float64ConstOp struct { } func (p projGTInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52022,12 +49936,6 @@ type projGTInt64DecimalConstOp struct { } func (p projGTInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52164,12 +50072,6 @@ type projGTFloat64Int16ConstOp struct { } func (p projGTFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52358,12 +50260,6 @@ type projGTFloat64Int32ConstOp struct { } func (p projGTFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52552,12 +50448,6 @@ type projGTFloat64Int64ConstOp struct { } func (p projGTFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52746,12 +50636,6 @@ type projGTFloat64Float64ConstOp struct { } func (p projGTFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52940,12 +50824,6 @@ type projGTFloat64DecimalConstOp struct { } func (p projGTFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53090,12 +50968,6 @@ type projGTTimestampTimestampConstOp struct { } func (p projGTTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53236,12 +51108,6 @@ type projGTIntervalIntervalConstOp struct { } func (p projGTIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53354,12 +51220,6 @@ type projGTJSONJSONConstOp struct { } func (p projGTJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53494,12 +51354,6 @@ type projGTDatumDatumConstOp struct { } func (p projGTDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53618,12 +51472,6 @@ type projGEBoolBoolConstOp struct { } func (p projGEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53768,12 +51616,6 @@ type projGEBytesBytesConstOp struct { } func (p projGEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53884,12 +51726,6 @@ type projGEDecimalInt16ConstOp struct { } func (p projGEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54026,12 +51862,6 @@ type projGEDecimalInt32ConstOp struct { } func (p projGEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54168,12 +51998,6 @@ type projGEDecimalInt64ConstOp struct { } func (p projGEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54310,12 +52134,6 @@ type projGEDecimalFloat64ConstOp struct { } func (p projGEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54460,12 +52278,6 @@ type projGEDecimalDecimalConstOp struct { } func (p projGEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54578,12 +52390,6 @@ type projGEInt16Int16ConstOp struct { } func (p projGEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54740,12 +52546,6 @@ type projGEInt16Int32ConstOp struct { } func (p projGEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54902,12 +52702,6 @@ type projGEInt16Int64ConstOp struct { } func (p projGEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55064,12 +52858,6 @@ type projGEInt16Float64ConstOp struct { } func (p projGEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55258,12 +53046,6 @@ type projGEInt16DecimalConstOp struct { } func (p projGEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55400,12 +53182,6 @@ type projGEInt32Int16ConstOp struct { } func (p projGEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55562,12 +53338,6 @@ type projGEInt32Int32ConstOp struct { } func (p projGEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55724,12 +53494,6 @@ type projGEInt32Int64ConstOp struct { } func (p projGEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55886,12 +53650,6 @@ type projGEInt32Float64ConstOp struct { } func (p projGEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56080,12 +53838,6 @@ type projGEInt32DecimalConstOp struct { } func (p projGEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56222,12 +53974,6 @@ type projGEInt64Int16ConstOp struct { } func (p projGEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56384,12 +54130,6 @@ type projGEInt64Int32ConstOp struct { } func (p projGEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56546,12 +54286,6 @@ type projGEInt64Int64ConstOp struct { } func (p projGEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56708,12 +54442,6 @@ type projGEInt64Float64ConstOp struct { } func (p projGEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56902,12 +54630,6 @@ type projGEInt64DecimalConstOp struct { } func (p projGEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57044,12 +54766,6 @@ type projGEFloat64Int16ConstOp struct { } func (p projGEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57238,12 +54954,6 @@ type projGEFloat64Int32ConstOp struct { } func (p projGEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57432,12 +55142,6 @@ type projGEFloat64Int64ConstOp struct { } func (p projGEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57626,12 +55330,6 @@ type projGEFloat64Float64ConstOp struct { } func (p projGEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57820,12 +55518,6 @@ type projGEFloat64DecimalConstOp struct { } func (p projGEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57970,12 +55662,6 @@ type projGETimestampTimestampConstOp struct { } func (p projGETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58116,12 +55802,6 @@ type projGEIntervalIntervalConstOp struct { } func (p projGEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58234,12 +55914,6 @@ type projGEJSONJSONConstOp struct { } func (p projGEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58374,12 +56048,6 @@ type projGEDatumDatumConstOp struct { } func (p projGEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58514,7 +56182,6 @@ func GetProjectionRConstOperator( allocator: allocator, colIdx: colIdx, outputIdx: outputIdx, - overloadHelper: execgen.OverloadHelper{BinFn: binFn, EvalCtx: evalCtx}, } c := colconv.GetDatumToPhysicalFn(constType)(constArg) leftType, rightType := inputTypes[colIdx], constType @@ -58530,21 +56197,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt16Int16ConstOp{ + op := &projBitandInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitandInt16Int32ConstOp{ + op := &projBitandInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitandInt16Int64ConstOp{ + op := &projBitandInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case 32: @@ -58552,21 +56222,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt32Int16ConstOp{ + op := &projBitandInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitandInt32Int32ConstOp{ + op := &projBitandInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitandInt32Int64ConstOp{ + op := &projBitandInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case -1: @@ -58575,21 +56248,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt64Int16ConstOp{ + op := &projBitandInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitandInt64Int32ConstOp{ + op := &projBitandInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitandInt64Int64ConstOp{ + op := &projBitandInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -58602,10 +56278,12 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projBitandDatumDatumConstOp{ + op := &projBitandDatumDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -58619,21 +56297,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt16Int16ConstOp{ + op := &projBitorInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitorInt16Int32ConstOp{ + op := &projBitorInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitorInt16Int64ConstOp{ + op := &projBitorInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case 32: @@ -58641,21 +56322,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt32Int16ConstOp{ + op := &projBitorInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitorInt32Int32ConstOp{ + op := &projBitorInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitorInt32Int64ConstOp{ + op := &projBitorInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case -1: @@ -58664,21 +56348,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt64Int16ConstOp{ + op := &projBitorInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitorInt64Int32ConstOp{ + op := &projBitorInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitorInt64Int64ConstOp{ + op := &projBitorInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -58691,10 +56378,12 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projBitorDatumDatumConstOp{ + op := &projBitorDatumDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -58708,21 +56397,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt16Int16ConstOp{ + op := &projBitxorInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitxorInt16Int32ConstOp{ + op := &projBitxorInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt16Int64ConstOp{ + op := &projBitxorInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case 32: @@ -58730,21 +56422,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt32Int16ConstOp{ + op := &projBitxorInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitxorInt32Int32ConstOp{ + op := &projBitxorInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt32Int64ConstOp{ + op := &projBitxorInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case -1: @@ -58753,21 +56448,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt64Int16ConstOp{ + op := &projBitxorInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projBitxorInt64Int32ConstOp{ + op := &projBitxorInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projBitxorInt64Int64ConstOp{ + op := &projBitxorInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -58780,10 +56478,12 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projBitxorDatumDatumConstOp{ + op := &projBitxorDatumDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -58798,30 +56498,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDecimalInt16ConstOp{ + op := &projPlusDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPlusDecimalInt32ConstOp{ + op := &projPlusDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPlusDecimalInt64ConstOp{ + op := &projPlusDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusDecimalDecimalConstOp{ + op := &projPlusDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -58832,39 +56536,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt16Int16ConstOp{ + op := &projPlusInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPlusInt16Int32ConstOp{ + op := &projPlusInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPlusInt16Int64ConstOp{ + op := &projPlusInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16DecimalConstOp{ + op := &projPlusInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16DatumConstOp{ + op := &projPlusInt16DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -58872,39 +56582,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt32Int16ConstOp{ + op := &projPlusInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPlusInt32Int32ConstOp{ + op := &projPlusInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPlusInt32Int64ConstOp{ + op := &projPlusInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32DecimalConstOp{ + op := &projPlusInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32DatumConstOp{ + op := &projPlusInt32DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -58913,39 +56629,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt64Int16ConstOp{ + op := &projPlusInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPlusInt64Int32ConstOp{ + op := &projPlusInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPlusInt64Int64ConstOp{ + op := &projPlusInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64DecimalConstOp{ + op := &projPlusInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64DatumConstOp{ + op := &projPlusInt64DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -58958,10 +56680,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projPlusFloat64Float64ConstOp{ + op := &projPlusFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -58974,10 +56697,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projPlusTimestampIntervalConstOp{ + op := &projPlusTimestampIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -58990,28 +56714,32 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projPlusIntervalTimestampConstOp{ + op := &projPlusIntervalTimestampConstOp{ projConstOpBase: projConstOpBase, constArg: c.(time.Time), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalIntervalConstOp{ + op := &projPlusIntervalIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalDatumConstOp{ + op := &projPlusIntervalDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -59024,29 +56752,37 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projPlusDatumIntervalConstOp{ + op := &projPlusDatumIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDatumInt16ConstOp{ + op := &projPlusDatumInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projPlusDatumInt32ConstOp{ + op := &projPlusDatumInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projPlusDatumInt64ConstOp{ + op := &projPlusDatumInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -59061,30 +56797,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDecimalInt16ConstOp{ + op := &projMinusDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusDecimalInt32ConstOp{ + op := &projMinusDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusDecimalInt64ConstOp{ + op := &projMinusDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusDecimalDecimalConstOp{ + op := &projMinusDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59095,39 +56835,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt16Int16ConstOp{ + op := &projMinusInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusInt16Int32ConstOp{ + op := &projMinusInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusInt16Int64ConstOp{ + op := &projMinusInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16DecimalConstOp{ + op := &projMinusInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16DatumConstOp{ + op := &projMinusInt16DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -59135,39 +56881,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt32Int16ConstOp{ + op := &projMinusInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusInt32Int32ConstOp{ + op := &projMinusInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusInt32Int64ConstOp{ + op := &projMinusInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32DecimalConstOp{ + op := &projMinusInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32DatumConstOp{ + op := &projMinusInt32DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -59176,39 +56928,45 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt64Int16ConstOp{ + op := &projMinusInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusInt64Int32ConstOp{ + op := &projMinusInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusInt64Int64ConstOp{ + op := &projMinusInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64DecimalConstOp{ + op := &projMinusInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64DatumConstOp{ + op := &projMinusInt64DatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -59221,10 +56979,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMinusFloat64Float64ConstOp{ + op := &projMinusFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -59237,19 +56996,21 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMinusTimestampTimestampConstOp{ + op := &projMinusTimestampTimestampConstOp{ projConstOpBase: projConstOpBase, constArg: c.(time.Time), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusTimestampIntervalConstOp{ + op := &projMinusTimestampIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -59262,19 +57023,22 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMinusIntervalIntervalConstOp{ + op := &projMinusIntervalIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusIntervalDatumConstOp{ + op := &projMinusIntervalDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -59287,29 +57051,33 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMinusJSONBytesConstOp{ + op := &projMinusJSONBytesConstOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusJSONInt16ConstOp{ + op := &projMinusJSONInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMinusJSONInt32ConstOp{ + op := &projMinusJSONInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMinusJSONInt64ConstOp{ + op := &projMinusJSONInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -59322,47 +57090,59 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMinusDatumDatumConstOp{ + op := &projMinusDatumDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumIntervalConstOp{ + op := &projMinusDatumIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.BytesFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumBytesConstOp{ + op := &projMinusDatumBytesConstOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDatumInt16ConstOp{ + op := &projMinusDatumInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projMinusDatumInt32ConstOp{ + op := &projMinusDatumInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projMinusDatumInt64ConstOp{ + op := &projMinusDatumInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -59377,39 +57157,44 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultDecimalInt16ConstOp{ + op := &projMultDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultDecimalInt32ConstOp{ + op := &projMultDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultDecimalInt64ConstOp{ + op := &projMultDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalDecimalConstOp{ + op := &projMultDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalIntervalConstOp{ + op := &projMultDecimalIntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -59420,39 +57205,44 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt16Int16ConstOp{ + op := &projMultInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultInt16Int32ConstOp{ + op := &projMultInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultInt16Int64ConstOp{ + op := &projMultInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16DecimalConstOp{ + op := &projMultInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16IntervalConstOp{ + op := &projMultInt16IntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } case 32: @@ -59460,39 +57250,44 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt32Int16ConstOp{ + op := &projMultInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultInt32Int32ConstOp{ + op := &projMultInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultInt32Int64ConstOp{ + op := &projMultInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32DecimalConstOp{ + op := &projMultInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32IntervalConstOp{ + op := &projMultInt32IntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } case -1: @@ -59501,39 +57296,44 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt64Int16ConstOp{ + op := &projMultInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultInt64Int32ConstOp{ + op := &projMultInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultInt64Int64ConstOp{ + op := &projMultInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64DecimalConstOp{ + op := &projMultInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64IntervalConstOp{ + op := &projMultInt64IntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -59546,19 +57346,21 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projMultFloat64Float64ConstOp{ + op := &projMultFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultFloat64IntervalConstOp{ + op := &projMultFloat64IntervalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(duration.Duration), - }, nil + } + return op, nil } } } @@ -59570,39 +57372,44 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultIntervalInt16ConstOp{ + op := &projMultIntervalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projMultIntervalInt32ConstOp{ + op := &projMultIntervalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projMultIntervalInt64ConstOp{ + op := &projMultIntervalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalFloat64ConstOp{ + op := &projMultIntervalFloat64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalDecimalConstOp{ + op := &projMultIntervalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59617,30 +57424,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivDecimalInt16ConstOp{ + op := &projDivDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projDivDecimalInt32ConstOp{ + op := &projDivDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projDivDecimalInt64ConstOp{ + op := &projDivDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivDecimalDecimalConstOp{ + op := &projDivDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59651,30 +57462,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt16Int16ConstOp{ + op := &projDivInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projDivInt16Int32ConstOp{ + op := &projDivInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projDivInt16Int64ConstOp{ + op := &projDivInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt16DecimalConstOp{ + op := &projDivInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case 32: @@ -59682,30 +57497,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt32Int16ConstOp{ + op := &projDivInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projDivInt32Int32ConstOp{ + op := &projDivInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projDivInt32Int64ConstOp{ + op := &projDivInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt32DecimalConstOp{ + op := &projDivInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case -1: @@ -59714,30 +57533,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt64Int16ConstOp{ + op := &projDivInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projDivInt64Int32ConstOp{ + op := &projDivInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projDivInt64Int64ConstOp{ + op := &projDivInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt64DecimalConstOp{ + op := &projDivInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59750,10 +57573,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projDivFloat64Float64ConstOp{ + op := &projDivFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -59766,19 +57590,21 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projDivIntervalInt64ConstOp{ + op := &projDivIntervalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projDivIntervalFloat64ConstOp{ + op := &projDivIntervalFloat64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -59793,30 +57619,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivDecimalInt16ConstOp{ + op := &projFloorDivDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projFloorDivDecimalInt32ConstOp{ + op := &projFloorDivDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projFloorDivDecimalInt64ConstOp{ + op := &projFloorDivDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivDecimalDecimalConstOp{ + op := &projFloorDivDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59827,30 +57657,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt16Int16ConstOp{ + op := &projFloorDivInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projFloorDivInt16Int32ConstOp{ + op := &projFloorDivInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt16Int64ConstOp{ + op := &projFloorDivInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt16DecimalConstOp{ + op := &projFloorDivInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case 32: @@ -59858,30 +57692,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt32Int16ConstOp{ + op := &projFloorDivInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projFloorDivInt32Int32ConstOp{ + op := &projFloorDivInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt32Int64ConstOp{ + op := &projFloorDivInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt32DecimalConstOp{ + op := &projFloorDivInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case -1: @@ -59890,30 +57728,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt64Int16ConstOp{ + op := &projFloorDivInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projFloorDivInt64Int32ConstOp{ + op := &projFloorDivInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projFloorDivInt64Int64ConstOp{ + op := &projFloorDivInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt64DecimalConstOp{ + op := &projFloorDivInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59926,10 +57768,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projFloorDivFloat64Float64ConstOp{ + op := &projFloorDivFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -59944,30 +57787,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModDecimalInt16ConstOp{ + op := &projModDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projModDecimalInt32ConstOp{ + op := &projModDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projModDecimalInt64ConstOp{ + op := &projModDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModDecimalDecimalConstOp{ + op := &projModDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -59978,30 +57825,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt16Int16ConstOp{ + op := &projModInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projModInt16Int32ConstOp{ + op := &projModInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projModInt16Int64ConstOp{ + op := &projModInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt16DecimalConstOp{ + op := &projModInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case 32: @@ -60009,30 +57860,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt32Int16ConstOp{ + op := &projModInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projModInt32Int32ConstOp{ + op := &projModInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projModInt32Int64ConstOp{ + op := &projModInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt32DecimalConstOp{ + op := &projModInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case -1: @@ -60041,30 +57896,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt64Int16ConstOp{ + op := &projModInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projModInt64Int32ConstOp{ + op := &projModInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projModInt64Int64ConstOp{ + op := &projModInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt64DecimalConstOp{ + op := &projModInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -60077,10 +57936,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projModFloat64Float64ConstOp{ + op := &projModFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -60095,30 +57955,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowDecimalInt16ConstOp{ + op := &projPowDecimalInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPowDecimalInt32ConstOp{ + op := &projPowDecimalInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPowDecimalInt64ConstOp{ + op := &projPowDecimalInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowDecimalDecimalConstOp{ + op := &projPowDecimalDecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -60129,30 +57993,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt16Int16ConstOp{ + op := &projPowInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPowInt16Int32ConstOp{ + op := &projPowInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPowInt16Int64ConstOp{ + op := &projPowInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt16DecimalConstOp{ + op := &projPowInt16DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case 32: @@ -60160,30 +58028,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt32Int16ConstOp{ + op := &projPowInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPowInt32Int32ConstOp{ + op := &projPowInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPowInt32Int64ConstOp{ + op := &projPowInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt32DecimalConstOp{ + op := &projPowInt32DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } case -1: @@ -60192,30 +58064,34 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt64Int16ConstOp{ + op := &projPowInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projPowInt64Int32ConstOp{ + op := &projPowInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projPowInt64Int64ConstOp{ + op := &projPowInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt64DecimalConstOp{ + op := &projPowInt64DecimalConstOp{ projConstOpBase: projConstOpBase, constArg: c.(apd.Decimal), - }, nil + } + return op, nil } } } @@ -60228,10 +58104,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projPowFloat64Float64ConstOp{ + op := &projPowFloat64Float64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(float64), - }, nil + } + return op, nil } } } @@ -60247,10 +58124,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projConcatBytesBytesConstOp{ + op := &projConcatBytesBytesConstOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + return op, nil } } } @@ -60263,10 +58141,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projConcatJSONJSONConstOp{ + op := &projConcatJSONJSONConstOp{ projConstOpBase: projConstOpBase, constArg: c.(json.JSON), - }, nil + } + return op, nil } } } @@ -60279,10 +58158,12 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projConcatDatumDatumConstOp{ + op := &projConcatDatumDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -60296,21 +58177,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt16Int16ConstOp{ + op := &projLShiftInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projLShiftInt16Int32ConstOp{ + op := &projLShiftInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt16Int64ConstOp{ + op := &projLShiftInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case 32: @@ -60318,21 +58202,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt32Int16ConstOp{ + op := &projLShiftInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projLShiftInt32Int32ConstOp{ + op := &projLShiftInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt32Int64ConstOp{ + op := &projLShiftInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case -1: @@ -60341,21 +58228,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt64Int16ConstOp{ + op := &projLShiftInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projLShiftInt64Int32ConstOp{ + op := &projLShiftInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projLShiftInt64Int64ConstOp{ + op := &projLShiftInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -60367,21 +58257,27 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftDatumInt16ConstOp{ + op := &projLShiftDatumInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projLShiftDatumInt32ConstOp{ + op := &projLShiftDatumInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projLShiftDatumInt64ConstOp{ + op := &projLShiftDatumInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -60395,21 +58291,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt16Int16ConstOp{ + op := &projRShiftInt16Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projRShiftInt16Int32ConstOp{ + op := &projRShiftInt16Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt16Int64ConstOp{ + op := &projRShiftInt16Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case 32: @@ -60417,21 +58316,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt32Int16ConstOp{ + op := &projRShiftInt32Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projRShiftInt32Int32ConstOp{ + op := &projRShiftInt32Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt32Int64ConstOp{ + op := &projRShiftInt32Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } case -1: @@ -60440,21 +58342,24 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt64Int16ConstOp{ + op := &projRShiftInt64Int16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projRShiftInt64Int32ConstOp{ + op := &projRShiftInt64Int32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projRShiftInt64Int64ConstOp{ + op := &projRShiftInt64Int64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -60466,21 +58371,27 @@ func GetProjectionRConstOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftDatumInt16ConstOp{ + op := &projRShiftDatumInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projRShiftDatumInt32ConstOp{ + op := &projRShiftDatumInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projRShiftDatumInt64ConstOp{ + op := &projRShiftDatumInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -60496,29 +58407,33 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValJSONBytesConstOp{ + op := &projJSONFetchValJSONBytesConstOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchValJSONInt16ConstOp{ + op := &projJSONFetchValJSONInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projJSONFetchValJSONInt32ConstOp{ + op := &projJSONFetchValJSONInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projJSONFetchValJSONInt64ConstOp{ + op := &projJSONFetchValJSONInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -60534,29 +58449,33 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextJSONBytesConstOp{ + op := &projJSONFetchTextJSONBytesConstOp{ projConstOpBase: projConstOpBase, constArg: c.([]byte), - }, nil + } + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchTextJSONInt16ConstOp{ + op := &projJSONFetchTextJSONInt16ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int16), - }, nil + } + return op, nil case 32: - return &projJSONFetchTextJSONInt32ConstOp{ + op := &projJSONFetchTextJSONInt32ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int32), - }, nil + } + return op, nil case -1: default: - return &projJSONFetchTextJSONInt64ConstOp{ + op := &projJSONFetchTextJSONInt64ConstOp{ projConstOpBase: projConstOpBase, constArg: c.(int64), - }, nil + } + return op, nil } } } @@ -60572,10 +58491,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValPathJSONDatumConstOp{ + op := &projJSONFetchValPathJSONDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + return op, nil } } } @@ -60591,10 +58511,11 @@ func GetProjectionRConstOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextPathJSONDatumConstOp{ + op := &projJSONFetchTextPathJSONDatumConstOp{ projConstOpBase: projConstOpBase, constArg: constArg, - }, nil + } + return op, nil } } } diff --git a/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go index fd6800d4d51e..e0ae8b248a42 100644 --- a/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go @@ -23,12 +23,6 @@ type projPrefixBytesBytesConstOp struct { } func (p projPrefixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -115,12 +109,6 @@ type projSuffixBytesBytesConstOp struct { } func (p projSuffixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -207,12 +195,6 @@ type projContainsBytesBytesConstOp struct { } func (p projContainsBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -299,12 +281,6 @@ type projRegexpBytesBytesConstOp struct { } func (p projRegexpBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -391,12 +367,6 @@ type projNotPrefixBytesBytesConstOp struct { } func (p projNotPrefixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -483,12 +453,6 @@ type projNotSuffixBytesBytesConstOp struct { } func (p projNotSuffixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -575,12 +539,6 @@ type projNotContainsBytesBytesConstOp struct { } func (p projNotContainsBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -667,12 +625,6 @@ type projNotRegexpBytesBytesConstOp struct { } func (p projNotRegexpBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index 81942161cd36..467d6f360bcc 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -51,20 +51,18 @@ var ( // around the problem we specify it here. type projConstOpBase struct { colexecop.OneInputHelper - allocator *colmem.Allocator - colIdx int - outputIdx int - overloadHelper execgen.OverloadHelper + allocator *colmem.Allocator + colIdx int + outputIdx int } // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { colexecop.OneInputHelper - allocator *colmem.Allocator - col1Idx int - col2Idx int - outputIdx int - overloadHelper execgen.OverloadHelper + allocator *colmem.Allocator + col1Idx int + col2Idx int + outputIdx int } type projBitandInt16Int16Op struct { @@ -72,12 +70,6 @@ type projBitandInt16Int16Op struct { } func (p projBitandInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -182,12 +174,6 @@ type projBitandInt16Int32Op struct { } func (p projBitandInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -292,12 +278,6 @@ type projBitandInt16Int64Op struct { } func (p projBitandInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -402,12 +382,6 @@ type projBitandInt32Int16Op struct { } func (p projBitandInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -512,12 +486,6 @@ type projBitandInt32Int32Op struct { } func (p projBitandInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -622,12 +590,6 @@ type projBitandInt32Int64Op struct { } func (p projBitandInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -732,12 +694,6 @@ type projBitandInt64Int16Op struct { } func (p projBitandInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -842,12 +798,6 @@ type projBitandInt64Int32Op struct { } func (p projBitandInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -952,12 +902,6 @@ type projBitandInt64Int64Op struct { } func (p projBitandInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1059,15 +1003,14 @@ func (p projBitandInt64Int64Op) Next() coldata.Batch { type projBitandDatumDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projBitandDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1196,12 +1139,6 @@ type projBitorInt16Int16Op struct { } func (p projBitorInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1306,12 +1243,6 @@ type projBitorInt16Int32Op struct { } func (p projBitorInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1416,12 +1347,6 @@ type projBitorInt16Int64Op struct { } func (p projBitorInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1526,12 +1451,6 @@ type projBitorInt32Int16Op struct { } func (p projBitorInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1636,12 +1555,6 @@ type projBitorInt32Int32Op struct { } func (p projBitorInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1746,12 +1659,6 @@ type projBitorInt32Int64Op struct { } func (p projBitorInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1856,12 +1763,6 @@ type projBitorInt64Int16Op struct { } func (p projBitorInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -1966,12 +1867,6 @@ type projBitorInt64Int32Op struct { } func (p projBitorInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2076,12 +1971,6 @@ type projBitorInt64Int64Op struct { } func (p projBitorInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2183,15 +2072,14 @@ func (p projBitorInt64Int64Op) Next() coldata.Batch { type projBitorDatumDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projBitorDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2320,12 +2208,6 @@ type projBitxorInt16Int16Op struct { } func (p projBitxorInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2430,12 +2312,6 @@ type projBitxorInt16Int32Op struct { } func (p projBitxorInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2540,12 +2416,6 @@ type projBitxorInt16Int64Op struct { } func (p projBitxorInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2650,12 +2520,6 @@ type projBitxorInt32Int16Op struct { } func (p projBitxorInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2760,12 +2624,6 @@ type projBitxorInt32Int32Op struct { } func (p projBitxorInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2870,12 +2728,6 @@ type projBitxorInt32Int64Op struct { } func (p projBitxorInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -2980,12 +2832,6 @@ type projBitxorInt64Int16Op struct { } func (p projBitxorInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3090,12 +2936,6 @@ type projBitxorInt64Int32Op struct { } func (p projBitxorInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3200,12 +3040,6 @@ type projBitxorInt64Int64Op struct { } func (p projBitxorInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3307,15 +3141,14 @@ func (p projBitxorInt64Int64Op) Next() coldata.Batch { type projBitxorDatumDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projBitxorDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3444,12 +3277,6 @@ type projPlusDecimalInt16Op struct { } func (p projPlusDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3582,12 +3409,6 @@ type projPlusDecimalInt32Op struct { } func (p projPlusDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3720,12 +3541,6 @@ type projPlusDecimalInt64Op struct { } func (p projPlusDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3858,12 +3673,6 @@ type projPlusDecimalDecimalOp struct { } func (p projPlusDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -3992,12 +3801,6 @@ type projPlusInt16Int16Op struct { } func (p projPlusInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4126,12 +3929,6 @@ type projPlusInt16Int32Op struct { } func (p projPlusInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4260,12 +4057,6 @@ type projPlusInt16Int64Op struct { } func (p projPlusInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4394,12 +4185,6 @@ type projPlusInt16DecimalOp struct { } func (p projPlusInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4533,15 +4318,14 @@ func (p projPlusInt16DecimalOp) Next() coldata.Batch { type projPlusInt16DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusInt16DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4688,12 +4472,6 @@ type projPlusInt32Int16Op struct { } func (p projPlusInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4822,12 +4600,6 @@ type projPlusInt32Int32Op struct { } func (p projPlusInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -4956,12 +4728,6 @@ type projPlusInt32Int64Op struct { } func (p projPlusInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5090,12 +4856,6 @@ type projPlusInt32DecimalOp struct { } func (p projPlusInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5229,15 +4989,14 @@ func (p projPlusInt32DecimalOp) Next() coldata.Batch { type projPlusInt32DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusInt32DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5384,12 +5143,6 @@ type projPlusInt64Int16Op struct { } func (p projPlusInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5518,12 +5271,6 @@ type projPlusInt64Int32Op struct { } func (p projPlusInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5652,12 +5399,6 @@ type projPlusInt64Int64Op struct { } func (p projPlusInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5786,12 +5527,6 @@ type projPlusInt64DecimalOp struct { } func (p projPlusInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -5925,15 +5660,14 @@ func (p projPlusInt64DecimalOp) Next() coldata.Batch { type projPlusInt64DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusInt64DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6080,12 +5814,6 @@ type projPlusFloat64Float64Op struct { } func (p projPlusFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6202,12 +5930,6 @@ type projPlusTimestampIntervalOp struct { } func (p projPlusTimestampIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6324,12 +6046,6 @@ type projPlusIntervalTimestampOp struct { } func (p projPlusIntervalTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6446,12 +6162,6 @@ type projPlusIntervalIntervalOp struct { } func (p projPlusIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6545,15 +6255,14 @@ func (p projPlusIntervalIntervalOp) Next() coldata.Batch { type projPlusIntervalDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusIntervalDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6697,15 +6406,14 @@ func (p projPlusIntervalDatumOp) Next() coldata.Batch { type projPlusDatumIntervalOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusDatumIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -6849,15 +6557,14 @@ func (p projPlusDatumIntervalOp) Next() coldata.Batch { type projPlusDatumInt16Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusDatumInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7001,15 +6708,14 @@ func (p projPlusDatumInt16Op) Next() coldata.Batch { type projPlusDatumInt32Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusDatumInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7153,15 +6859,14 @@ func (p projPlusDatumInt32Op) Next() coldata.Batch { type projPlusDatumInt64Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projPlusDatumInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7308,12 +7013,6 @@ type projMinusDecimalInt16Op struct { } func (p projMinusDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7446,12 +7145,6 @@ type projMinusDecimalInt32Op struct { } func (p projMinusDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7584,12 +7277,6 @@ type projMinusDecimalInt64Op struct { } func (p projMinusDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7722,12 +7409,6 @@ type projMinusDecimalDecimalOp struct { } func (p projMinusDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7856,12 +7537,6 @@ type projMinusInt16Int16Op struct { } func (p projMinusInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -7990,12 +7665,6 @@ type projMinusInt16Int32Op struct { } func (p projMinusInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8124,12 +7793,6 @@ type projMinusInt16Int64Op struct { } func (p projMinusInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8258,12 +7921,6 @@ type projMinusInt16DecimalOp struct { } func (p projMinusInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8397,15 +8054,14 @@ func (p projMinusInt16DecimalOp) Next() coldata.Batch { type projMinusInt16DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusInt16DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8552,12 +8208,6 @@ type projMinusInt32Int16Op struct { } func (p projMinusInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8686,12 +8336,6 @@ type projMinusInt32Int32Op struct { } func (p projMinusInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8820,12 +8464,6 @@ type projMinusInt32Int64Op struct { } func (p projMinusInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -8954,12 +8592,6 @@ type projMinusInt32DecimalOp struct { } func (p projMinusInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9093,15 +8725,14 @@ func (p projMinusInt32DecimalOp) Next() coldata.Batch { type projMinusInt32DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusInt32DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9248,12 +8879,6 @@ type projMinusInt64Int16Op struct { } func (p projMinusInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9382,12 +9007,6 @@ type projMinusInt64Int32Op struct { } func (p projMinusInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9516,12 +9135,6 @@ type projMinusInt64Int64Op struct { } func (p projMinusInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9650,12 +9263,6 @@ type projMinusInt64DecimalOp struct { } func (p projMinusInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9789,15 +9396,14 @@ func (p projMinusInt64DecimalOp) Next() coldata.Batch { type projMinusInt64DatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusInt64DatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -9944,12 +9550,6 @@ type projMinusFloat64Float64Op struct { } func (p projMinusFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10066,12 +9666,6 @@ type projMinusTimestampTimestampOp struct { } func (p projMinusTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10180,12 +9774,6 @@ type projMinusTimestampIntervalOp struct { } func (p projMinusTimestampIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10302,12 +9890,6 @@ type projMinusIntervalIntervalOp struct { } func (p projMinusIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10401,15 +9983,14 @@ func (p projMinusIntervalIntervalOp) Next() coldata.Batch { type projMinusIntervalDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusIntervalDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10556,12 +10137,6 @@ type projMinusJSONBytesOp struct { } func (p projMinusJSONBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10690,12 +10265,6 @@ type projMinusJSONInt16Op struct { } func (p projMinusJSONInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10810,12 +10379,6 @@ type projMinusJSONInt32Op struct { } func (p projMinusJSONInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -10930,12 +10493,6 @@ type projMinusJSONInt64Op struct { } func (p projMinusJSONInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11047,15 +10604,14 @@ func (p projMinusJSONInt64Op) Next() coldata.Batch { type projMinusDatumDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11181,15 +10737,14 @@ func (p projMinusDatumDatumOp) Next() coldata.Batch { type projMinusDatumIntervalOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11333,15 +10888,14 @@ func (p projMinusDatumIntervalOp) Next() coldata.Batch { type projMinusDatumBytesOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11483,15 +11037,14 @@ func (p projMinusDatumBytesOp) Next() coldata.Batch { type projMinusDatumInt16Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11635,15 +11188,14 @@ func (p projMinusDatumInt16Op) Next() coldata.Batch { type projMinusDatumInt32Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11787,15 +11339,14 @@ func (p projMinusDatumInt32Op) Next() coldata.Batch { type projMinusDatumInt64Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projMinusDatumInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -11942,12 +11493,6 @@ type projMultDecimalInt16Op struct { } func (p projMultDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12080,12 +11625,6 @@ type projMultDecimalInt32Op struct { } func (p projMultDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12218,12 +11757,6 @@ type projMultDecimalInt64Op struct { } func (p projMultDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12356,12 +11889,6 @@ type projMultDecimalDecimalOp struct { } func (p projMultDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12490,12 +12017,6 @@ type projMultDecimalIntervalOp struct { } func (p projMultDecimalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12612,12 +12133,6 @@ type projMultInt16Int16Op struct { } func (p projMultInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12778,12 +12293,6 @@ type projMultInt16Int32Op struct { } func (p projMultInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -12944,12 +12453,6 @@ type projMultInt16Int64Op struct { } func (p projMultInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13110,12 +12613,6 @@ type projMultInt16DecimalOp struct { } func (p projMultInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13252,12 +12749,6 @@ type projMultInt16IntervalOp struct { } func (p projMultInt16IntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13354,12 +12845,6 @@ type projMultInt32Int16Op struct { } func (p projMultInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13520,12 +13005,6 @@ type projMultInt32Int32Op struct { } func (p projMultInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13686,12 +13165,6 @@ type projMultInt32Int64Op struct { } func (p projMultInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13852,12 +13325,6 @@ type projMultInt32DecimalOp struct { } func (p projMultInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -13994,12 +13461,6 @@ type projMultInt32IntervalOp struct { } func (p projMultInt32IntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14096,12 +13557,6 @@ type projMultInt64Int16Op struct { } func (p projMultInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14262,12 +13717,6 @@ type projMultInt64Int32Op struct { } func (p projMultInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14428,12 +13877,6 @@ type projMultInt64Int64Op struct { } func (p projMultInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14594,12 +14037,6 @@ type projMultInt64DecimalOp struct { } func (p projMultInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14736,12 +14173,6 @@ type projMultInt64IntervalOp struct { } func (p projMultInt64IntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14838,12 +14269,6 @@ type projMultFloat64Float64Op struct { } func (p projMultFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -14960,12 +14385,6 @@ type projMultFloat64IntervalOp struct { } func (p projMultFloat64IntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15062,12 +14481,6 @@ type projMultIntervalInt16Op struct { } func (p projMultIntervalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15164,12 +14577,6 @@ type projMultIntervalInt32Op struct { } func (p projMultIntervalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15266,12 +14673,6 @@ type projMultIntervalInt64Op struct { } func (p projMultIntervalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15368,12 +14769,6 @@ type projMultIntervalFloat64Op struct { } func (p projMultIntervalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15470,12 +14865,6 @@ type projMultIntervalDecimalOp struct { } func (p projMultIntervalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15592,12 +14981,6 @@ type projDivDecimalInt16Op struct { } func (p projDivDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15746,12 +15129,6 @@ type projDivDecimalInt32Op struct { } func (p projDivDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -15900,12 +15277,6 @@ type projDivDecimalInt64Op struct { } func (p projDivDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16054,12 +15425,6 @@ type projDivDecimalDecimalOp struct { } func (p projDivDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16204,12 +15569,6 @@ type projDivInt16Int16Op struct { } func (p projDivInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16354,12 +15713,6 @@ type projDivInt16Int32Op struct { } func (p projDivInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16504,12 +15857,6 @@ type projDivInt16Int64Op struct { } func (p projDivInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16654,12 +16001,6 @@ type projDivInt16DecimalOp struct { } func (p projDivInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16812,12 +16153,6 @@ type projDivInt32Int16Op struct { } func (p projDivInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -16962,12 +16297,6 @@ type projDivInt32Int32Op struct { } func (p projDivInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17112,12 +16441,6 @@ type projDivInt32Int64Op struct { } func (p projDivInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17262,12 +16585,6 @@ type projDivInt32DecimalOp struct { } func (p projDivInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17420,12 +16737,6 @@ type projDivInt64Int16Op struct { } func (p projDivInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17570,12 +16881,6 @@ type projDivInt64Int32Op struct { } func (p projDivInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17720,12 +17025,6 @@ type projDivInt64Int64Op struct { } func (p projDivInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -17870,12 +17169,6 @@ type projDivInt64DecimalOp struct { } func (p projDivInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18028,12 +17321,6 @@ type projDivFloat64Float64Op struct { } func (p projDivFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18166,12 +17453,6 @@ type projDivIntervalInt64Op struct { } func (p projDivIntervalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18284,12 +17565,6 @@ type projDivIntervalFloat64Op struct { } func (p projDivIntervalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18402,12 +17677,6 @@ type projFloorDivDecimalInt16Op struct { } func (p projFloorDivDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18556,12 +17825,6 @@ type projFloorDivDecimalInt32Op struct { } func (p projFloorDivDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18710,12 +17973,6 @@ type projFloorDivDecimalInt64Op struct { } func (p projFloorDivDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -18864,12 +18121,6 @@ type projFloorDivDecimalDecimalOp struct { } func (p projFloorDivDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19014,12 +18265,6 @@ type projFloorDivInt16Int16Op struct { } func (p projFloorDivInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19144,12 +18389,6 @@ type projFloorDivInt16Int32Op struct { } func (p projFloorDivInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19274,12 +18513,6 @@ type projFloorDivInt16Int64Op struct { } func (p projFloorDivInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19404,12 +18637,6 @@ type projFloorDivInt16DecimalOp struct { } func (p projFloorDivInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19562,12 +18789,6 @@ type projFloorDivInt32Int16Op struct { } func (p projFloorDivInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19692,12 +18913,6 @@ type projFloorDivInt32Int32Op struct { } func (p projFloorDivInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19822,12 +19037,6 @@ type projFloorDivInt32Int64Op struct { } func (p projFloorDivInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -19952,12 +19161,6 @@ type projFloorDivInt32DecimalOp struct { } func (p projFloorDivInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20110,12 +19313,6 @@ type projFloorDivInt64Int16Op struct { } func (p projFloorDivInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20240,12 +19437,6 @@ type projFloorDivInt64Int32Op struct { } func (p projFloorDivInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20370,12 +19561,6 @@ type projFloorDivInt64Int64Op struct { } func (p projFloorDivInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20500,12 +19685,6 @@ type projFloorDivInt64DecimalOp struct { } func (p projFloorDivInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20658,12 +19837,6 @@ type projFloorDivFloat64Float64Op struct { } func (p projFloorDivFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20796,12 +19969,6 @@ type projModDecimalInt16Op struct { } func (p projModDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -20950,12 +20117,6 @@ type projModDecimalInt32Op struct { } func (p projModDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21104,12 +20265,6 @@ type projModDecimalInt64Op struct { } func (p projModDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21258,12 +20413,6 @@ type projModDecimalDecimalOp struct { } func (p projModDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21408,12 +20557,6 @@ type projModInt16Int16Op struct { } func (p projModInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21538,12 +20681,6 @@ type projModInt16Int32Op struct { } func (p projModInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21668,12 +20805,6 @@ type projModInt16Int64Op struct { } func (p projModInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21798,12 +20929,6 @@ type projModInt16DecimalOp struct { } func (p projModInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -21956,12 +21081,6 @@ type projModInt32Int16Op struct { } func (p projModInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22086,12 +21205,6 @@ type projModInt32Int32Op struct { } func (p projModInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22216,12 +21329,6 @@ type projModInt32Int64Op struct { } func (p projModInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22346,12 +21453,6 @@ type projModInt32DecimalOp struct { } func (p projModInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22504,12 +21605,6 @@ type projModInt64Int16Op struct { } func (p projModInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22634,12 +21729,6 @@ type projModInt64Int32Op struct { } func (p projModInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22764,12 +21853,6 @@ type projModInt64Int64Op struct { } func (p projModInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -22894,12 +21977,6 @@ type projModInt64DecimalOp struct { } func (p projModInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23052,12 +22129,6 @@ type projModFloat64Float64Op struct { } func (p projModFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23190,12 +22261,6 @@ type projPowDecimalInt16Op struct { } func (p projPowDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23328,12 +22393,6 @@ type projPowDecimalInt32Op struct { } func (p projPowDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23466,12 +22525,6 @@ type projPowDecimalInt64Op struct { } func (p projPowDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23604,12 +22657,6 @@ type projPowDecimalDecimalOp struct { } func (p projPowDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23738,12 +22785,6 @@ type projPowInt16Int16Op struct { } func (p projPowInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -23896,12 +22937,6 @@ type projPowInt16Int32Op struct { } func (p projPowInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24054,12 +23089,6 @@ type projPowInt16Int64Op struct { } func (p projPowInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24212,12 +23241,6 @@ type projPowInt16DecimalOp struct { } func (p projPowInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24354,12 +23377,6 @@ type projPowInt32Int16Op struct { } func (p projPowInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24512,12 +23529,6 @@ type projPowInt32Int32Op struct { } func (p projPowInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24670,12 +23681,6 @@ type projPowInt32Int64Op struct { } func (p projPowInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24828,12 +23833,6 @@ type projPowInt32DecimalOp struct { } func (p projPowInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -24970,12 +23969,6 @@ type projPowInt64Int16Op struct { } func (p projPowInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25128,12 +24121,6 @@ type projPowInt64Int32Op struct { } func (p projPowInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25286,12 +24273,6 @@ type projPowInt64Int64Op struct { } func (p projPowInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25444,12 +24425,6 @@ type projPowInt64DecimalOp struct { } func (p projPowInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25586,12 +24561,6 @@ type projPowFloat64Float64Op struct { } func (p projPowFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25708,12 +24677,6 @@ type projConcatBytesBytesOp struct { } func (p projConcatBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25834,12 +24797,6 @@ type projConcatJSONJSONOp struct { } func (p projConcatJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -25953,15 +24910,14 @@ func (p projConcatJSONJSONOp) Next() coldata.Batch { type projConcatDatumDatumOp struct { projOpBase + execgen.BinaryOverloadHelper } func (p projConcatDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26090,12 +25046,6 @@ type projLShiftInt16Int16Op struct { } func (p projLShiftInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26224,12 +25174,6 @@ type projLShiftInt16Int32Op struct { } func (p projLShiftInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26358,12 +25302,6 @@ type projLShiftInt16Int64Op struct { } func (p projLShiftInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26492,12 +25430,6 @@ type projLShiftInt32Int16Op struct { } func (p projLShiftInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26626,12 +25558,6 @@ type projLShiftInt32Int32Op struct { } func (p projLShiftInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26760,12 +25686,6 @@ type projLShiftInt32Int64Op struct { } func (p projLShiftInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -26894,12 +25814,6 @@ type projLShiftInt64Int16Op struct { } func (p projLShiftInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27028,12 +25942,6 @@ type projLShiftInt64Int32Op struct { } func (p projLShiftInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27162,12 +26070,6 @@ type projLShiftInt64Int64Op struct { } func (p projLShiftInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27293,15 +26195,14 @@ func (p projLShiftInt64Int64Op) Next() coldata.Batch { type projLShiftDatumInt16Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projLShiftDatumInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27445,15 +26346,14 @@ func (p projLShiftDatumInt16Op) Next() coldata.Batch { type projLShiftDatumInt32Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projLShiftDatumInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27597,15 +26497,14 @@ func (p projLShiftDatumInt32Op) Next() coldata.Batch { type projLShiftDatumInt64Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projLShiftDatumInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27752,12 +26651,6 @@ type projRShiftInt16Int16Op struct { } func (p projRShiftInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -27886,12 +26779,6 @@ type projRShiftInt16Int32Op struct { } func (p projRShiftInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28020,12 +26907,6 @@ type projRShiftInt16Int64Op struct { } func (p projRShiftInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28154,12 +27035,6 @@ type projRShiftInt32Int16Op struct { } func (p projRShiftInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28288,12 +27163,6 @@ type projRShiftInt32Int32Op struct { } func (p projRShiftInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28422,12 +27291,6 @@ type projRShiftInt32Int64Op struct { } func (p projRShiftInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28556,12 +27419,6 @@ type projRShiftInt64Int16Op struct { } func (p projRShiftInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28690,12 +27547,6 @@ type projRShiftInt64Int32Op struct { } func (p projRShiftInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28824,12 +27675,6 @@ type projRShiftInt64Int64Op struct { } func (p projRShiftInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -28955,15 +27800,14 @@ func (p projRShiftInt64Int64Op) Next() coldata.Batch { type projRShiftDatumInt16Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projRShiftDatumInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29107,15 +27951,14 @@ func (p projRShiftDatumInt16Op) Next() coldata.Batch { type projRShiftDatumInt32Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projRShiftDatumInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29259,15 +28102,14 @@ func (p projRShiftDatumInt32Op) Next() coldata.Batch { type projRShiftDatumInt64Op struct { projOpBase + execgen.BinaryOverloadHelper } func (p projRShiftDatumInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29414,12 +28256,6 @@ type projJSONFetchValJSONBytesOp struct { } func (p projJSONFetchValJSONBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29560,12 +28396,6 @@ type projJSONFetchValJSONInt16Op struct { } func (p projJSONFetchValJSONInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29696,12 +28526,6 @@ type projJSONFetchValJSONInt32Op struct { } func (p projJSONFetchValJSONInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29832,12 +28656,6 @@ type projJSONFetchValJSONInt64Op struct { } func (p projJSONFetchValJSONInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -29968,12 +28786,6 @@ type projJSONFetchTextJSONBytesOp struct { } func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30150,12 +28962,6 @@ type projJSONFetchTextJSONInt16Op struct { } func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30322,12 +29128,6 @@ type projJSONFetchTextJSONInt32Op struct { } func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30494,12 +29294,6 @@ type projJSONFetchTextJSONInt64Op struct { } func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30666,12 +29460,6 @@ type projJSONFetchValPathJSONDatumOp struct { } func (p projJSONFetchValPathJSONDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30800,12 +29588,6 @@ type projJSONFetchTextPathJSONDatumOp struct { } func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -30974,12 +29756,6 @@ type projEQBoolBoolOp struct { } func (p projEQBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31132,12 +29908,6 @@ type projEQBytesBytesOp struct { } func (p projEQBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31254,12 +30024,6 @@ type projEQDecimalInt16Op struct { } func (p projEQDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31404,12 +30168,6 @@ type projEQDecimalInt32Op struct { } func (p projEQDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31554,12 +30312,6 @@ type projEQDecimalInt64Op struct { } func (p projEQDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31704,12 +30456,6 @@ type projEQDecimalFloat64Op struct { } func (p projEQDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31862,12 +30608,6 @@ type projEQDecimalDecimalOp struct { } func (p projEQDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -31988,12 +30728,6 @@ type projEQInt16Int16Op struct { } func (p projEQInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32158,12 +30892,6 @@ type projEQInt16Int32Op struct { } func (p projEQInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32328,12 +31056,6 @@ type projEQInt16Int64Op struct { } func (p projEQInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32498,12 +31220,6 @@ type projEQInt16Float64Op struct { } func (p projEQInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32700,12 +31416,6 @@ type projEQInt16DecimalOp struct { } func (p projEQInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -32850,12 +31560,6 @@ type projEQInt32Int16Op struct { } func (p projEQInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33020,12 +31724,6 @@ type projEQInt32Int32Op struct { } func (p projEQInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33190,12 +31888,6 @@ type projEQInt32Int64Op struct { } func (p projEQInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33360,12 +32052,6 @@ type projEQInt32Float64Op struct { } func (p projEQInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33562,12 +32248,6 @@ type projEQInt32DecimalOp struct { } func (p projEQInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33712,12 +32392,6 @@ type projEQInt64Int16Op struct { } func (p projEQInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -33882,12 +32556,6 @@ type projEQInt64Int32Op struct { } func (p projEQInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34052,12 +32720,6 @@ type projEQInt64Int64Op struct { } func (p projEQInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34222,12 +32884,6 @@ type projEQInt64Float64Op struct { } func (p projEQInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34424,12 +33080,6 @@ type projEQInt64DecimalOp struct { } func (p projEQInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34574,12 +33224,6 @@ type projEQFloat64Int16Op struct { } func (p projEQFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34776,12 +33420,6 @@ type projEQFloat64Int32Op struct { } func (p projEQFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -34978,12 +33616,6 @@ type projEQFloat64Int64Op struct { } func (p projEQFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35180,12 +33812,6 @@ type projEQFloat64Float64Op struct { } func (p projEQFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35382,12 +34008,6 @@ type projEQFloat64DecimalOp struct { } func (p projEQFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35540,12 +34160,6 @@ type projEQTimestampTimestampOp struct { } func (p projEQTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35694,12 +34308,6 @@ type projEQIntervalIntervalOp struct { } func (p projEQIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35820,12 +34428,6 @@ type projEQJSONJSONOp struct { } func (p projEQJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -35966,12 +34568,6 @@ type projEQDatumDatumOp struct { } func (p projEQDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36096,12 +34692,6 @@ type projNEBoolBoolOp struct { } func (p projNEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36254,12 +34844,6 @@ type projNEBytesBytesOp struct { } func (p projNEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36376,12 +34960,6 @@ type projNEDecimalInt16Op struct { } func (p projNEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36526,12 +35104,6 @@ type projNEDecimalInt32Op struct { } func (p projNEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36676,12 +35248,6 @@ type projNEDecimalInt64Op struct { } func (p projNEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36826,12 +35392,6 @@ type projNEDecimalFloat64Op struct { } func (p projNEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -36984,12 +35544,6 @@ type projNEDecimalDecimalOp struct { } func (p projNEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37110,12 +35664,6 @@ type projNEInt16Int16Op struct { } func (p projNEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37280,12 +35828,6 @@ type projNEInt16Int32Op struct { } func (p projNEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37450,12 +35992,6 @@ type projNEInt16Int64Op struct { } func (p projNEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37620,12 +36156,6 @@ type projNEInt16Float64Op struct { } func (p projNEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37822,12 +36352,6 @@ type projNEInt16DecimalOp struct { } func (p projNEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -37972,12 +36496,6 @@ type projNEInt32Int16Op struct { } func (p projNEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38142,12 +36660,6 @@ type projNEInt32Int32Op struct { } func (p projNEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38312,12 +36824,6 @@ type projNEInt32Int64Op struct { } func (p projNEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38482,12 +36988,6 @@ type projNEInt32Float64Op struct { } func (p projNEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38684,12 +37184,6 @@ type projNEInt32DecimalOp struct { } func (p projNEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -38834,12 +37328,6 @@ type projNEInt64Int16Op struct { } func (p projNEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39004,12 +37492,6 @@ type projNEInt64Int32Op struct { } func (p projNEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39174,12 +37656,6 @@ type projNEInt64Int64Op struct { } func (p projNEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39344,12 +37820,6 @@ type projNEInt64Float64Op struct { } func (p projNEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39546,12 +38016,6 @@ type projNEInt64DecimalOp struct { } func (p projNEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39696,12 +38160,6 @@ type projNEFloat64Int16Op struct { } func (p projNEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -39898,12 +38356,6 @@ type projNEFloat64Int32Op struct { } func (p projNEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40100,12 +38552,6 @@ type projNEFloat64Int64Op struct { } func (p projNEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40302,12 +38748,6 @@ type projNEFloat64Float64Op struct { } func (p projNEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40504,12 +38944,6 @@ type projNEFloat64DecimalOp struct { } func (p projNEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40662,12 +39096,6 @@ type projNETimestampTimestampOp struct { } func (p projNETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40816,12 +39244,6 @@ type projNEIntervalIntervalOp struct { } func (p projNEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -40942,12 +39364,6 @@ type projNEJSONJSONOp struct { } func (p projNEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41088,12 +39504,6 @@ type projNEDatumDatumOp struct { } func (p projNEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41218,12 +39628,6 @@ type projLTBoolBoolOp struct { } func (p projLTBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41376,12 +39780,6 @@ type projLTBytesBytesOp struct { } func (p projLTBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41498,12 +39896,6 @@ type projLTDecimalInt16Op struct { } func (p projLTDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41648,12 +40040,6 @@ type projLTDecimalInt32Op struct { } func (p projLTDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41798,12 +40184,6 @@ type projLTDecimalInt64Op struct { } func (p projLTDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -41948,12 +40328,6 @@ type projLTDecimalFloat64Op struct { } func (p projLTDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42106,12 +40480,6 @@ type projLTDecimalDecimalOp struct { } func (p projLTDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42232,12 +40600,6 @@ type projLTInt16Int16Op struct { } func (p projLTInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42402,12 +40764,6 @@ type projLTInt16Int32Op struct { } func (p projLTInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42572,12 +40928,6 @@ type projLTInt16Int64Op struct { } func (p projLTInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42742,12 +41092,6 @@ type projLTInt16Float64Op struct { } func (p projLTInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -42944,12 +41288,6 @@ type projLTInt16DecimalOp struct { } func (p projLTInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43094,12 +41432,6 @@ type projLTInt32Int16Op struct { } func (p projLTInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43264,12 +41596,6 @@ type projLTInt32Int32Op struct { } func (p projLTInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43434,12 +41760,6 @@ type projLTInt32Int64Op struct { } func (p projLTInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43604,12 +41924,6 @@ type projLTInt32Float64Op struct { } func (p projLTInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43806,12 +42120,6 @@ type projLTInt32DecimalOp struct { } func (p projLTInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -43956,12 +42264,6 @@ type projLTInt64Int16Op struct { } func (p projLTInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44126,12 +42428,6 @@ type projLTInt64Int32Op struct { } func (p projLTInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44296,12 +42592,6 @@ type projLTInt64Int64Op struct { } func (p projLTInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44466,12 +42756,6 @@ type projLTInt64Float64Op struct { } func (p projLTInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44668,12 +42952,6 @@ type projLTInt64DecimalOp struct { } func (p projLTInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -44818,12 +43096,6 @@ type projLTFloat64Int16Op struct { } func (p projLTFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45020,12 +43292,6 @@ type projLTFloat64Int32Op struct { } func (p projLTFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45222,12 +43488,6 @@ type projLTFloat64Int64Op struct { } func (p projLTFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45424,12 +43684,6 @@ type projLTFloat64Float64Op struct { } func (p projLTFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45626,12 +43880,6 @@ type projLTFloat64DecimalOp struct { } func (p projLTFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45784,12 +44032,6 @@ type projLTTimestampTimestampOp struct { } func (p projLTTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -45938,12 +44180,6 @@ type projLTIntervalIntervalOp struct { } func (p projLTIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46064,12 +44300,6 @@ type projLTJSONJSONOp struct { } func (p projLTJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46210,12 +44440,6 @@ type projLTDatumDatumOp struct { } func (p projLTDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46340,12 +44564,6 @@ type projLEBoolBoolOp struct { } func (p projLEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46498,12 +44716,6 @@ type projLEBytesBytesOp struct { } func (p projLEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46620,12 +44832,6 @@ type projLEDecimalInt16Op struct { } func (p projLEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46770,12 +44976,6 @@ type projLEDecimalInt32Op struct { } func (p projLEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -46920,12 +45120,6 @@ type projLEDecimalInt64Op struct { } func (p projLEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47070,12 +45264,6 @@ type projLEDecimalFloat64Op struct { } func (p projLEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47228,12 +45416,6 @@ type projLEDecimalDecimalOp struct { } func (p projLEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47354,12 +45536,6 @@ type projLEInt16Int16Op struct { } func (p projLEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47524,12 +45700,6 @@ type projLEInt16Int32Op struct { } func (p projLEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47694,12 +45864,6 @@ type projLEInt16Int64Op struct { } func (p projLEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -47864,12 +46028,6 @@ type projLEInt16Float64Op struct { } func (p projLEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48066,12 +46224,6 @@ type projLEInt16DecimalOp struct { } func (p projLEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48216,12 +46368,6 @@ type projLEInt32Int16Op struct { } func (p projLEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48386,12 +46532,6 @@ type projLEInt32Int32Op struct { } func (p projLEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48556,12 +46696,6 @@ type projLEInt32Int64Op struct { } func (p projLEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48726,12 +46860,6 @@ type projLEInt32Float64Op struct { } func (p projLEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -48928,12 +47056,6 @@ type projLEInt32DecimalOp struct { } func (p projLEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49078,12 +47200,6 @@ type projLEInt64Int16Op struct { } func (p projLEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49248,12 +47364,6 @@ type projLEInt64Int32Op struct { } func (p projLEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49418,12 +47528,6 @@ type projLEInt64Int64Op struct { } func (p projLEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49588,12 +47692,6 @@ type projLEInt64Float64Op struct { } func (p projLEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49790,12 +47888,6 @@ type projLEInt64DecimalOp struct { } func (p projLEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -49940,12 +48032,6 @@ type projLEFloat64Int16Op struct { } func (p projLEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50142,12 +48228,6 @@ type projLEFloat64Int32Op struct { } func (p projLEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50344,12 +48424,6 @@ type projLEFloat64Int64Op struct { } func (p projLEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50546,12 +48620,6 @@ type projLEFloat64Float64Op struct { } func (p projLEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50748,12 +48816,6 @@ type projLEFloat64DecimalOp struct { } func (p projLEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -50906,12 +48968,6 @@ type projLETimestampTimestampOp struct { } func (p projLETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51060,12 +49116,6 @@ type projLEIntervalIntervalOp struct { } func (p projLEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51186,12 +49236,6 @@ type projLEJSONJSONOp struct { } func (p projLEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51332,12 +49376,6 @@ type projLEDatumDatumOp struct { } func (p projLEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51462,12 +49500,6 @@ type projGTBoolBoolOp struct { } func (p projGTBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51620,12 +49652,6 @@ type projGTBytesBytesOp struct { } func (p projGTBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51742,12 +49768,6 @@ type projGTDecimalInt16Op struct { } func (p projGTDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -51892,12 +49912,6 @@ type projGTDecimalInt32Op struct { } func (p projGTDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52042,12 +50056,6 @@ type projGTDecimalInt64Op struct { } func (p projGTDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52192,12 +50200,6 @@ type projGTDecimalFloat64Op struct { } func (p projGTDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52350,12 +50352,6 @@ type projGTDecimalDecimalOp struct { } func (p projGTDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52476,12 +50472,6 @@ type projGTInt16Int16Op struct { } func (p projGTInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52646,12 +50636,6 @@ type projGTInt16Int32Op struct { } func (p projGTInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52816,12 +50800,6 @@ type projGTInt16Int64Op struct { } func (p projGTInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -52986,12 +50964,6 @@ type projGTInt16Float64Op struct { } func (p projGTInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53188,12 +51160,6 @@ type projGTInt16DecimalOp struct { } func (p projGTInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53338,12 +51304,6 @@ type projGTInt32Int16Op struct { } func (p projGTInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53508,12 +51468,6 @@ type projGTInt32Int32Op struct { } func (p projGTInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53678,12 +51632,6 @@ type projGTInt32Int64Op struct { } func (p projGTInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -53848,12 +51796,6 @@ type projGTInt32Float64Op struct { } func (p projGTInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54050,12 +51992,6 @@ type projGTInt32DecimalOp struct { } func (p projGTInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54200,12 +52136,6 @@ type projGTInt64Int16Op struct { } func (p projGTInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54370,12 +52300,6 @@ type projGTInt64Int32Op struct { } func (p projGTInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54540,12 +52464,6 @@ type projGTInt64Int64Op struct { } func (p projGTInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54710,12 +52628,6 @@ type projGTInt64Float64Op struct { } func (p projGTInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -54912,12 +52824,6 @@ type projGTInt64DecimalOp struct { } func (p projGTInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55062,12 +52968,6 @@ type projGTFloat64Int16Op struct { } func (p projGTFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55264,12 +53164,6 @@ type projGTFloat64Int32Op struct { } func (p projGTFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55466,12 +53360,6 @@ type projGTFloat64Int64Op struct { } func (p projGTFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55668,12 +53556,6 @@ type projGTFloat64Float64Op struct { } func (p projGTFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -55870,12 +53752,6 @@ type projGTFloat64DecimalOp struct { } func (p projGTFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56028,12 +53904,6 @@ type projGTTimestampTimestampOp struct { } func (p projGTTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56182,12 +54052,6 @@ type projGTIntervalIntervalOp struct { } func (p projGTIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56308,12 +54172,6 @@ type projGTJSONJSONOp struct { } func (p projGTJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56454,12 +54312,6 @@ type projGTDatumDatumOp struct { } func (p projGTDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56584,12 +54436,6 @@ type projGEBoolBoolOp struct { } func (p projGEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56742,12 +54588,6 @@ type projGEBytesBytesOp struct { } func (p projGEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -56864,12 +54704,6 @@ type projGEDecimalInt16Op struct { } func (p projGEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57014,12 +54848,6 @@ type projGEDecimalInt32Op struct { } func (p projGEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57164,12 +54992,6 @@ type projGEDecimalInt64Op struct { } func (p projGEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57314,12 +55136,6 @@ type projGEDecimalFloat64Op struct { } func (p projGEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57472,12 +55288,6 @@ type projGEDecimalDecimalOp struct { } func (p projGEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57598,12 +55408,6 @@ type projGEInt16Int16Op struct { } func (p projGEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57768,12 +55572,6 @@ type projGEInt16Int32Op struct { } func (p projGEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -57938,12 +55736,6 @@ type projGEInt16Int64Op struct { } func (p projGEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58108,12 +55900,6 @@ type projGEInt16Float64Op struct { } func (p projGEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58310,12 +56096,6 @@ type projGEInt16DecimalOp struct { } func (p projGEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58460,12 +56240,6 @@ type projGEInt32Int16Op struct { } func (p projGEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58630,12 +56404,6 @@ type projGEInt32Int32Op struct { } func (p projGEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58800,12 +56568,6 @@ type projGEInt32Int64Op struct { } func (p projGEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -58970,12 +56732,6 @@ type projGEInt32Float64Op struct { } func (p projGEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -59172,12 +56928,6 @@ type projGEInt32DecimalOp struct { } func (p projGEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -59322,12 +57072,6 @@ type projGEInt64Int16Op struct { } func (p projGEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -59492,12 +57236,6 @@ type projGEInt64Int32Op struct { } func (p projGEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -59662,12 +57400,6 @@ type projGEInt64Int64Op struct { } func (p projGEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -59832,12 +57564,6 @@ type projGEInt64Float64Op struct { } func (p projGEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60034,12 +57760,6 @@ type projGEInt64DecimalOp struct { } func (p projGEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60184,12 +57904,6 @@ type projGEFloat64Int16Op struct { } func (p projGEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60386,12 +58100,6 @@ type projGEFloat64Int32Op struct { } func (p projGEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60588,12 +58296,6 @@ type projGEFloat64Int64Op struct { } func (p projGEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60790,12 +58492,6 @@ type projGEFloat64Float64Op struct { } func (p projGEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -60992,12 +58688,6 @@ type projGEFloat64DecimalOp struct { } func (p projGEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -61150,12 +58840,6 @@ type projGETimestampTimestampOp struct { } func (p projGETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -61304,12 +58988,6 @@ type projGEIntervalIntervalOp struct { } func (p projGEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -61430,12 +59108,6 @@ type projGEJSONJSONOp struct { } func (p projGEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -61576,12 +59248,6 @@ type projGEDatumDatumOp struct { } func (p projGEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -61723,7 +59389,6 @@ func GetProjectionOperator( col1Idx: col1Idx, col2Idx: col2Idx, outputIdx: outputIdx, - overloadHelper: execgen.OverloadHelper{BinFn: binFn, EvalCtx: evalCtx}, } leftType, rightType := inputTypes[col1Idx], inputTypes[col2Idx] @@ -61739,12 +59404,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt16Int16Op{projOpBase: projOpBase}, nil + op := &projBitandInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitandInt16Int32Op{projOpBase: projOpBase}, nil + op := &projBitandInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitandInt16Int64Op{projOpBase: projOpBase}, nil + op := &projBitandInt16Int64Op{projOpBase: projOpBase} + return op, nil } } case 32: @@ -61752,12 +59420,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt32Int16Op{projOpBase: projOpBase}, nil + op := &projBitandInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitandInt32Int32Op{projOpBase: projOpBase}, nil + op := &projBitandInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitandInt32Int64Op{projOpBase: projOpBase}, nil + op := &projBitandInt32Int64Op{projOpBase: projOpBase} + return op, nil } } case -1: @@ -61766,12 +59437,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitandInt64Int16Op{projOpBase: projOpBase}, nil + op := &projBitandInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitandInt64Int32Op{projOpBase: projOpBase}, nil + op := &projBitandInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitandInt64Int64Op{projOpBase: projOpBase}, nil + op := &projBitandInt64Int64Op{projOpBase: projOpBase} + return op, nil } } } @@ -61784,7 +59458,9 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projBitandDatumDatumOp{projOpBase: projOpBase}, nil + op := &projBitandDatumDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -61798,12 +59474,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt16Int16Op{projOpBase: projOpBase}, nil + op := &projBitorInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitorInt16Int32Op{projOpBase: projOpBase}, nil + op := &projBitorInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitorInt16Int64Op{projOpBase: projOpBase}, nil + op := &projBitorInt16Int64Op{projOpBase: projOpBase} + return op, nil } } case 32: @@ -61811,12 +59490,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt32Int16Op{projOpBase: projOpBase}, nil + op := &projBitorInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitorInt32Int32Op{projOpBase: projOpBase}, nil + op := &projBitorInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitorInt32Int64Op{projOpBase: projOpBase}, nil + op := &projBitorInt32Int64Op{projOpBase: projOpBase} + return op, nil } } case -1: @@ -61825,12 +59507,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitorInt64Int16Op{projOpBase: projOpBase}, nil + op := &projBitorInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitorInt64Int32Op{projOpBase: projOpBase}, nil + op := &projBitorInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitorInt64Int64Op{projOpBase: projOpBase}, nil + op := &projBitorInt64Int64Op{projOpBase: projOpBase} + return op, nil } } } @@ -61843,7 +59528,9 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projBitorDatumDatumOp{projOpBase: projOpBase}, nil + op := &projBitorDatumDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -61857,12 +59544,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt16Int16Op{projOpBase: projOpBase}, nil + op := &projBitxorInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitxorInt16Int32Op{projOpBase: projOpBase}, nil + op := &projBitxorInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitxorInt16Int64Op{projOpBase: projOpBase}, nil + op := &projBitxorInt16Int64Op{projOpBase: projOpBase} + return op, nil } } case 32: @@ -61870,12 +59560,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt32Int16Op{projOpBase: projOpBase}, nil + op := &projBitxorInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitxorInt32Int32Op{projOpBase: projOpBase}, nil + op := &projBitxorInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitxorInt32Int64Op{projOpBase: projOpBase}, nil + op := &projBitxorInt32Int64Op{projOpBase: projOpBase} + return op, nil } } case -1: @@ -61884,12 +59577,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projBitxorInt64Int16Op{projOpBase: projOpBase}, nil + op := &projBitxorInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projBitxorInt64Int32Op{projOpBase: projOpBase}, nil + op := &projBitxorInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projBitxorInt64Int64Op{projOpBase: projOpBase}, nil + op := &projBitxorInt64Int64Op{projOpBase: projOpBase} + return op, nil } } } @@ -61902,7 +59598,9 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projBitxorDatumDatumOp{projOpBase: projOpBase}, nil + op := &projBitxorDatumDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -61917,18 +59615,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projPlusDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPlusDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projPlusDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPlusDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projPlusDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projPlusDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -61939,24 +59641,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt16Int16Op{projOpBase: projOpBase}, nil + op := &projPlusInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPlusInt16Int32Op{projOpBase: projOpBase}, nil + op := &projPlusInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPlusInt16Int64Op{projOpBase: projOpBase}, nil + op := &projPlusInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projPlusInt16DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt16DatumOp{projOpBase: projOpBase}, nil + op := &projPlusInt16DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -61964,24 +59672,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt32Int16Op{projOpBase: projOpBase}, nil + op := &projPlusInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPlusInt32Int32Op{projOpBase: projOpBase}, nil + op := &projPlusInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPlusInt32Int64Op{projOpBase: projOpBase}, nil + op := &projPlusInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projPlusInt32DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt32DatumOp{projOpBase: projOpBase}, nil + op := &projPlusInt32DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -61990,24 +59704,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusInt64Int16Op{projOpBase: projOpBase}, nil + op := &projPlusInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPlusInt64Int32Op{projOpBase: projOpBase}, nil + op := &projPlusInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPlusInt64Int64Op{projOpBase: projOpBase}, nil + op := &projPlusInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projPlusInt64DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusInt64DatumOp{projOpBase: projOpBase}, nil + op := &projPlusInt64DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62020,7 +59740,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projPlusFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projPlusFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62033,7 +59754,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projPlusTimestampIntervalOp{projOpBase: projOpBase}, nil + op := &projPlusTimestampIntervalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62046,19 +59768,23 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projPlusIntervalTimestampOp{projOpBase: projOpBase}, nil + op := &projPlusIntervalTimestampOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalIntervalOp{projOpBase: projOpBase}, nil + op := &projPlusIntervalIntervalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projPlusIntervalDatumOp{projOpBase: projOpBase}, nil + op := &projPlusIntervalDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62071,17 +59797,25 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projPlusDatumIntervalOp{projOpBase: projOpBase}, nil + op := &projPlusDatumIntervalOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projPlusDatumInt16Op{projOpBase: projOpBase}, nil + op := &projPlusDatumInt16Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projPlusDatumInt32Op{projOpBase: projOpBase}, nil + op := &projPlusDatumInt32Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projPlusDatumInt64Op{projOpBase: projOpBase}, nil + op := &projPlusDatumInt64Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62096,18 +59830,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projMinusDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMinusDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projMinusDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMinusDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projMinusDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projMinusDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62118,24 +59856,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt16Int16Op{projOpBase: projOpBase}, nil + op := &projMinusInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMinusInt16Int32Op{projOpBase: projOpBase}, nil + op := &projMinusInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMinusInt16Int64Op{projOpBase: projOpBase}, nil + op := &projMinusInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projMinusInt16DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt16DatumOp{projOpBase: projOpBase}, nil + op := &projMinusInt16DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case 32: @@ -62143,24 +59887,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt32Int16Op{projOpBase: projOpBase}, nil + op := &projMinusInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMinusInt32Int32Op{projOpBase: projOpBase}, nil + op := &projMinusInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMinusInt32Int64Op{projOpBase: projOpBase}, nil + op := &projMinusInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projMinusInt32DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt32DatumOp{projOpBase: projOpBase}, nil + op := &projMinusInt32DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } case -1: @@ -62169,24 +59919,30 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusInt64Int16Op{projOpBase: projOpBase}, nil + op := &projMinusInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMinusInt64Int32Op{projOpBase: projOpBase}, nil + op := &projMinusInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMinusInt64Int64Op{projOpBase: projOpBase}, nil + op := &projMinusInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projMinusInt64DecimalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusInt64DatumOp{projOpBase: projOpBase}, nil + op := &projMinusInt64DatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62199,7 +59955,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMinusFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projMinusFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62212,13 +59969,15 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMinusTimestampTimestampOp{projOpBase: projOpBase}, nil + op := &projMinusTimestampTimestampOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusTimestampIntervalOp{projOpBase: projOpBase}, nil + op := &projMinusTimestampIntervalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62231,13 +59990,16 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMinusIntervalIntervalOp{projOpBase: projOpBase}, nil + op := &projMinusIntervalIntervalOp{projOpBase: projOpBase} + return op, nil } case typeconv.DatumVecCanonicalTypeFamily: switch rightType.Width() { case -1: default: - return &projMinusIntervalDatumOp{projOpBase: projOpBase}, nil + op := &projMinusIntervalDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62250,17 +60012,21 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMinusJSONBytesOp{projOpBase: projOpBase}, nil + op := &projMinusJSONBytesOp{projOpBase: projOpBase} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusJSONInt16Op{projOpBase: projOpBase}, nil + op := &projMinusJSONInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMinusJSONInt32Op{projOpBase: projOpBase}, nil + op := &projMinusJSONInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMinusJSONInt64Op{projOpBase: projOpBase}, nil + op := &projMinusJSONInt64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62273,29 +60039,41 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMinusDatumDatumOp{projOpBase: projOpBase}, nil + op := &projMinusDatumDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumIntervalOp{projOpBase: projOpBase}, nil + op := &projMinusDatumIntervalOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.BytesFamily: switch rightType.Width() { case -1: default: - return &projMinusDatumBytesOp{projOpBase: projOpBase}, nil + op := &projMinusDatumBytesOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projMinusDatumInt16Op{projOpBase: projOpBase}, nil + op := &projMinusDatumInt16Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projMinusDatumInt32Op{projOpBase: projOpBase}, nil + op := &projMinusDatumInt32Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projMinusDatumInt64Op{projOpBase: projOpBase}, nil + op := &projMinusDatumInt64Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62310,24 +60088,29 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projMultDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMultDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projMultDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMultDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projMultDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projMultDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultDecimalIntervalOp{projOpBase: projOpBase}, nil + op := &projMultDecimalIntervalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62338,24 +60121,29 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt16Int16Op{projOpBase: projOpBase}, nil + op := &projMultInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMultInt16Int32Op{projOpBase: projOpBase}, nil + op := &projMultInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMultInt16Int64Op{projOpBase: projOpBase}, nil + op := &projMultInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projMultInt16DecimalOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt16IntervalOp{projOpBase: projOpBase}, nil + op := &projMultInt16IntervalOp{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62363,24 +60151,29 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt32Int16Op{projOpBase: projOpBase}, nil + op := &projMultInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMultInt32Int32Op{projOpBase: projOpBase}, nil + op := &projMultInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMultInt32Int64Op{projOpBase: projOpBase}, nil + op := &projMultInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projMultInt32DecimalOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt32IntervalOp{projOpBase: projOpBase}, nil + op := &projMultInt32IntervalOp{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62389,24 +60182,29 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultInt64Int16Op{projOpBase: projOpBase}, nil + op := &projMultInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMultInt64Int32Op{projOpBase: projOpBase}, nil + op := &projMultInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMultInt64Int64Op{projOpBase: projOpBase}, nil + op := &projMultInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projMultInt64DecimalOp{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultInt64IntervalOp{projOpBase: projOpBase}, nil + op := &projMultInt64IntervalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62419,13 +60217,15 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projMultFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projMultFloat64Float64Op{projOpBase: projOpBase} + return op, nil } case types.IntervalFamily: switch rightType.Width() { case -1: default: - return &projMultFloat64IntervalOp{projOpBase: projOpBase}, nil + op := &projMultFloat64IntervalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62437,24 +60237,29 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projMultIntervalInt16Op{projOpBase: projOpBase}, nil + op := &projMultIntervalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projMultIntervalInt32Op{projOpBase: projOpBase}, nil + op := &projMultIntervalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projMultIntervalInt64Op{projOpBase: projOpBase}, nil + op := &projMultIntervalInt64Op{projOpBase: projOpBase} + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalFloat64Op{projOpBase: projOpBase}, nil + op := &projMultIntervalFloat64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projMultIntervalDecimalOp{projOpBase: projOpBase}, nil + op := &projMultIntervalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62469,18 +60274,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projDivDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projDivDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projDivDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projDivDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projDivDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projDivDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62491,18 +60300,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt16Int16Op{projOpBase: projOpBase}, nil + op := &projDivInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projDivInt16Int32Op{projOpBase: projOpBase}, nil + op := &projDivInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projDivInt16Int64Op{projOpBase: projOpBase}, nil + op := &projDivInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projDivInt16DecimalOp{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62510,18 +60323,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt32Int16Op{projOpBase: projOpBase}, nil + op := &projDivInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projDivInt32Int32Op{projOpBase: projOpBase}, nil + op := &projDivInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projDivInt32Int64Op{projOpBase: projOpBase}, nil + op := &projDivInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projDivInt32DecimalOp{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62530,18 +60347,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projDivInt64Int16Op{projOpBase: projOpBase}, nil + op := &projDivInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projDivInt64Int32Op{projOpBase: projOpBase}, nil + op := &projDivInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projDivInt64Int64Op{projOpBase: projOpBase}, nil + op := &projDivInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projDivInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projDivInt64DecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62554,7 +60375,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projDivFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projDivFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62567,13 +60389,15 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projDivIntervalInt64Op{projOpBase: projOpBase}, nil + op := &projDivIntervalInt64Op{projOpBase: projOpBase} + return op, nil } case types.FloatFamily: switch rightType.Width() { case -1: default: - return &projDivIntervalFloat64Op{projOpBase: projOpBase}, nil + op := &projDivIntervalFloat64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62588,18 +60412,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projFloorDivDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projFloorDivDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projFloorDivDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projFloorDivDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projFloorDivDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projFloorDivDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62610,18 +60438,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt16Int16Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projFloorDivInt16Int32Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projFloorDivInt16Int64Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projFloorDivInt16DecimalOp{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62629,18 +60461,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt32Int16Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projFloorDivInt32Int32Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projFloorDivInt32Int64Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projFloorDivInt32DecimalOp{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62649,18 +60485,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projFloorDivInt64Int16Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projFloorDivInt64Int32Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projFloorDivInt64Int64Op{projOpBase: projOpBase}, nil + op := &projFloorDivInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projFloorDivInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projFloorDivInt64DecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62673,7 +60513,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projFloorDivFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projFloorDivFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62688,18 +60529,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projModDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projModDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projModDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projModDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projModDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projModDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62710,18 +60555,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt16Int16Op{projOpBase: projOpBase}, nil + op := &projModInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projModInt16Int32Op{projOpBase: projOpBase}, nil + op := &projModInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projModInt16Int64Op{projOpBase: projOpBase}, nil + op := &projModInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projModInt16DecimalOp{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62729,18 +60578,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt32Int16Op{projOpBase: projOpBase}, nil + op := &projModInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projModInt32Int32Op{projOpBase: projOpBase}, nil + op := &projModInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projModInt32Int64Op{projOpBase: projOpBase}, nil + op := &projModInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projModInt32DecimalOp{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62749,18 +60602,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projModInt64Int16Op{projOpBase: projOpBase}, nil + op := &projModInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projModInt64Int32Op{projOpBase: projOpBase}, nil + op := &projModInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projModInt64Int64Op{projOpBase: projOpBase}, nil + op := &projModInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projModInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projModInt64DecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62773,7 +60630,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projModFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projModFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62788,18 +60646,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowDecimalInt16Op{projOpBase: projOpBase}, nil + op := &projPowDecimalInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPowDecimalInt32Op{projOpBase: projOpBase}, nil + op := &projPowDecimalInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPowDecimalInt64Op{projOpBase: projOpBase}, nil + op := &projPowDecimalInt64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowDecimalDecimalOp{projOpBase: projOpBase}, nil + op := &projPowDecimalDecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62810,18 +60672,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt16Int16Op{projOpBase: projOpBase}, nil + op := &projPowInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPowInt16Int32Op{projOpBase: projOpBase}, nil + op := &projPowInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPowInt16Int64Op{projOpBase: projOpBase}, nil + op := &projPowInt16Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt16DecimalOp{projOpBase: projOpBase}, nil + op := &projPowInt16DecimalOp{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62829,18 +60695,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt32Int16Op{projOpBase: projOpBase}, nil + op := &projPowInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPowInt32Int32Op{projOpBase: projOpBase}, nil + op := &projPowInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPowInt32Int64Op{projOpBase: projOpBase}, nil + op := &projPowInt32Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt32DecimalOp{projOpBase: projOpBase}, nil + op := &projPowInt32DecimalOp{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62849,18 +60719,22 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projPowInt64Int16Op{projOpBase: projOpBase}, nil + op := &projPowInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projPowInt64Int32Op{projOpBase: projOpBase}, nil + op := &projPowInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projPowInt64Int64Op{projOpBase: projOpBase}, nil + op := &projPowInt64Int64Op{projOpBase: projOpBase} + return op, nil } case types.DecimalFamily: switch rightType.Width() { case -1: default: - return &projPowInt64DecimalOp{projOpBase: projOpBase}, nil + op := &projPowInt64DecimalOp{projOpBase: projOpBase} + return op, nil } } } @@ -62873,7 +60747,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projPowFloat64Float64Op{projOpBase: projOpBase}, nil + op := &projPowFloat64Float64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62889,7 +60764,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projConcatBytesBytesOp{projOpBase: projOpBase}, nil + op := &projConcatBytesBytesOp{projOpBase: projOpBase} + return op, nil } } } @@ -62902,7 +60778,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projConcatJSONJSONOp{projOpBase: projOpBase}, nil + op := &projConcatJSONJSONOp{projOpBase: projOpBase} + return op, nil } } } @@ -62915,7 +60792,9 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projConcatDatumDatumOp{projOpBase: projOpBase}, nil + op := &projConcatDatumDatumOp{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62929,12 +60808,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt16Int16Op{projOpBase: projOpBase}, nil + op := &projLShiftInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projLShiftInt16Int32Op{projOpBase: projOpBase}, nil + op := &projLShiftInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projLShiftInt16Int64Op{projOpBase: projOpBase}, nil + op := &projLShiftInt16Int64Op{projOpBase: projOpBase} + return op, nil } } case 32: @@ -62942,12 +60824,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt32Int16Op{projOpBase: projOpBase}, nil + op := &projLShiftInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projLShiftInt32Int32Op{projOpBase: projOpBase}, nil + op := &projLShiftInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projLShiftInt32Int64Op{projOpBase: projOpBase}, nil + op := &projLShiftInt32Int64Op{projOpBase: projOpBase} + return op, nil } } case -1: @@ -62956,12 +60841,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftInt64Int16Op{projOpBase: projOpBase}, nil + op := &projLShiftInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projLShiftInt64Int32Op{projOpBase: projOpBase}, nil + op := &projLShiftInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projLShiftInt64Int64Op{projOpBase: projOpBase}, nil + op := &projLShiftInt64Int64Op{projOpBase: projOpBase} + return op, nil } } } @@ -62973,12 +60861,18 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projLShiftDatumInt16Op{projOpBase: projOpBase}, nil + op := &projLShiftDatumInt16Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projLShiftDatumInt32Op{projOpBase: projOpBase}, nil + op := &projLShiftDatumInt32Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projLShiftDatumInt64Op{projOpBase: projOpBase}, nil + op := &projLShiftDatumInt64Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -62992,12 +60886,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt16Int16Op{projOpBase: projOpBase}, nil + op := &projRShiftInt16Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projRShiftInt16Int32Op{projOpBase: projOpBase}, nil + op := &projRShiftInt16Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projRShiftInt16Int64Op{projOpBase: projOpBase}, nil + op := &projRShiftInt16Int64Op{projOpBase: projOpBase} + return op, nil } } case 32: @@ -63005,12 +60902,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt32Int16Op{projOpBase: projOpBase}, nil + op := &projRShiftInt32Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projRShiftInt32Int32Op{projOpBase: projOpBase}, nil + op := &projRShiftInt32Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projRShiftInt32Int64Op{projOpBase: projOpBase}, nil + op := &projRShiftInt32Int64Op{projOpBase: projOpBase} + return op, nil } } case -1: @@ -63019,12 +60919,15 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftInt64Int16Op{projOpBase: projOpBase}, nil + op := &projRShiftInt64Int16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projRShiftInt64Int32Op{projOpBase: projOpBase}, nil + op := &projRShiftInt64Int32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projRShiftInt64Int64Op{projOpBase: projOpBase}, nil + op := &projRShiftInt64Int64Op{projOpBase: projOpBase} + return op, nil } } } @@ -63036,12 +60939,18 @@ func GetProjectionOperator( case types.IntFamily: switch rightType.Width() { case 16: - return &projRShiftDatumInt16Op{projOpBase: projOpBase}, nil + op := &projRShiftDatumInt16Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case 32: - return &projRShiftDatumInt32Op{projOpBase: projOpBase}, nil + op := &projRShiftDatumInt32Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil case -1: default: - return &projRShiftDatumInt64Op{projOpBase: projOpBase}, nil + op := &projRShiftDatumInt64Op{projOpBase: projOpBase} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + return op, nil } } } @@ -63057,17 +60966,21 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValJSONBytesOp{projOpBase: projOpBase}, nil + op := &projJSONFetchValJSONBytesOp{projOpBase: projOpBase} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchValJSONInt16Op{projOpBase: projOpBase}, nil + op := &projJSONFetchValJSONInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projJSONFetchValJSONInt32Op{projOpBase: projOpBase}, nil + op := &projJSONFetchValJSONInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projJSONFetchValJSONInt64Op{projOpBase: projOpBase}, nil + op := &projJSONFetchValJSONInt64Op{projOpBase: projOpBase} + return op, nil } } } @@ -63083,17 +60996,21 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextJSONBytesOp{projOpBase: projOpBase}, nil + op := &projJSONFetchTextJSONBytesOp{projOpBase: projOpBase} + return op, nil } case types.IntFamily: switch rightType.Width() { case 16: - return &projJSONFetchTextJSONInt16Op{projOpBase: projOpBase}, nil + op := &projJSONFetchTextJSONInt16Op{projOpBase: projOpBase} + return op, nil case 32: - return &projJSONFetchTextJSONInt32Op{projOpBase: projOpBase}, nil + op := &projJSONFetchTextJSONInt32Op{projOpBase: projOpBase} + return op, nil case -1: default: - return &projJSONFetchTextJSONInt64Op{projOpBase: projOpBase}, nil + op := &projJSONFetchTextJSONInt64Op{projOpBase: projOpBase} + return op, nil } } } @@ -63109,7 +61026,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchValPathJSONDatumOp{projOpBase: projOpBase}, nil + op := &projJSONFetchValPathJSONDatumOp{projOpBase: projOpBase} + return op, nil } } } @@ -63125,7 +61043,8 @@ func GetProjectionOperator( switch rightType.Width() { case -1: default: - return &projJSONFetchTextPathJSONDatumOp{projOpBase: projOpBase}, nil + op := &projJSONFetchTextPathJSONDatumOp{projOpBase: projOpBase} + return op, nil } } } diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index 10a163eaca38..57c139f8148a 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -81,35 +81,36 @@ func _ASSIGN(_, _, _, _, _, _ interface{}) { // around the problem we specify it here. type projConstOpBase struct { colexecop.OneInputHelper - allocator *colmem.Allocator - colIdx int - outputIdx int - overloadHelper execgen.OverloadHelper + allocator *colmem.Allocator + colIdx int + outputIdx int } // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { colexecop.OneInputHelper - allocator *colmem.Allocator - col1Idx int - col2Idx int - outputIdx int - overloadHelper execgen.OverloadHelper + allocator *colmem.Allocator + col1Idx int + col2Idx int + outputIdx int } // {{define "projOp"}} type _OP_NAME struct { projOpBase + // {{if .NeedsBinaryOverloadHelper}} + execgen.BinaryOverloadHelper + // {{end}} } func (p _OP_NAME) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the projection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper + // {{if .NeedsBinaryOverloadHelper}} + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. + _overloadHelper := p.BinaryOverloadHelper + // {{end}} batch := p.Input.Next() n := batch.Length() if n == 0 { @@ -264,7 +265,6 @@ func GetProjectionOperator( col1Idx: col1Idx, col2Idx: col2Idx, outputIdx: outputIdx, - overloadHelper: execgen.OverloadHelper{BinFn: binFn, EvalCtx: evalCtx}, } leftType, rightType := inputTypes[col1Idx], inputTypes[col2Idx] @@ -285,7 +285,11 @@ func GetProjectionOperator( switch rightType.Width() { // {{range .RightWidths}} case _RIGHT_TYPE_WIDTH: - return &_OP_NAME{projOpBase: projOpBase}, nil + op := &_OP_NAME{projOpBase: projOpBase} + // {{if .NeedsBinaryOverloadHelper}} + op.BinaryOverloadHelper = execgen.BinaryOverloadHelper{BinFn: binFn, EvalCtx: evalCtx} + // {{end}} + return op, nil // {{end}} } // {{end}} diff --git a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go index b42a68cd0157..f01e2d913dae 100644 --- a/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/sel_like_ops.eg.go @@ -23,12 +23,6 @@ type selPrefixBytesBytesConstOp struct { } func (p *selPrefixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -112,12 +106,6 @@ type selSuffixBytesBytesConstOp struct { } func (p *selSuffixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -201,12 +189,6 @@ type selContainsBytesBytesConstOp struct { } func (p *selContainsBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -290,12 +272,6 @@ type selRegexpBytesBytesConstOp struct { } func (p *selRegexpBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -379,12 +355,6 @@ type selNotPrefixBytesBytesConstOp struct { } func (p *selNotPrefixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -468,12 +438,6 @@ type selNotSuffixBytesBytesConstOp struct { } func (p *selNotSuffixBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -557,12 +521,6 @@ type selNotContainsBytesBytesConstOp struct { } func (p *selNotContainsBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -646,12 +604,6 @@ type selNotRegexpBytesBytesConstOp struct { } func (p *selNotRegexpBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { diff --git a/pkg/sql/colexec/colexecsel/selection_ops.eg.go b/pkg/sql/colexec/colexecsel/selection_ops.eg.go index 595e725cfd57..31209f014543 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops.eg.go +++ b/pkg/sql/colexec/colexecsel/selection_ops.eg.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -42,16 +41,14 @@ var ( // constant, except for the constant itself. type selConstOpBase struct { colexecop.OneInputHelper - colIdx int - overloadHelper execgen.OverloadHelper + colIdx int } // selOpBase contains all of the fields for non-constant binary selections. type selOpBase struct { colexecop.OneInputHelper - col1Idx int - col2Idx int - overloadHelper execgen.OverloadHelper + col1Idx int + col2Idx int } type selEQBoolBoolConstOp struct { @@ -60,12 +57,6 @@ type selEQBoolBoolConstOp struct { } func (p *selEQBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -206,12 +197,6 @@ type selEQBoolBoolOp struct { } func (p *selEQBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -364,12 +349,6 @@ type selEQBytesBytesConstOp struct { } func (p *selEQBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -476,12 +455,6 @@ type selEQBytesBytesOp struct { } func (p *selEQBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -598,12 +571,6 @@ type selEQDecimalInt16ConstOp struct { } func (p *selEQDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -736,12 +703,6 @@ type selEQDecimalInt16Op struct { } func (p *selEQDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -886,12 +847,6 @@ type selEQDecimalInt32ConstOp struct { } func (p *selEQDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1024,12 +979,6 @@ type selEQDecimalInt32Op struct { } func (p *selEQDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1174,12 +1123,6 @@ type selEQDecimalInt64ConstOp struct { } func (p *selEQDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1312,12 +1255,6 @@ type selEQDecimalInt64Op struct { } func (p *selEQDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1462,12 +1399,6 @@ type selEQDecimalFloat64ConstOp struct { } func (p *selEQDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1608,12 +1539,6 @@ type selEQDecimalFloat64Op struct { } func (p *selEQDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1766,12 +1691,6 @@ type selEQDecimalDecimalConstOp struct { } func (p *selEQDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -1880,12 +1799,6 @@ type selEQDecimalDecimalOp struct { } func (p *selEQDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2006,12 +1919,6 @@ type selEQInt16Int16ConstOp struct { } func (p *selEQInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2164,12 +2071,6 @@ type selEQInt16Int16Op struct { } func (p *selEQInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2334,12 +2235,6 @@ type selEQInt16Int32ConstOp struct { } func (p *selEQInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2492,12 +2387,6 @@ type selEQInt16Int32Op struct { } func (p *selEQInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2662,12 +2551,6 @@ type selEQInt16Int64ConstOp struct { } func (p *selEQInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2820,12 +2703,6 @@ type selEQInt16Int64Op struct { } func (p *selEQInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -2990,12 +2867,6 @@ type selEQInt16Float64ConstOp struct { } func (p *selEQInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3180,12 +3051,6 @@ type selEQInt16Float64Op struct { } func (p *selEQInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3382,12 +3247,6 @@ type selEQInt16DecimalConstOp struct { } func (p *selEQInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3520,12 +3379,6 @@ type selEQInt16DecimalOp struct { } func (p *selEQInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3670,12 +3523,6 @@ type selEQInt32Int16ConstOp struct { } func (p *selEQInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3828,12 +3675,6 @@ type selEQInt32Int16Op struct { } func (p *selEQInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -3998,12 +3839,6 @@ type selEQInt32Int32ConstOp struct { } func (p *selEQInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -4156,12 +3991,6 @@ type selEQInt32Int32Op struct { } func (p *selEQInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -4326,12 +4155,6 @@ type selEQInt32Int64ConstOp struct { } func (p *selEQInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -4484,12 +4307,6 @@ type selEQInt32Int64Op struct { } func (p *selEQInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -4654,12 +4471,6 @@ type selEQInt32Float64ConstOp struct { } func (p *selEQInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -4844,12 +4655,6 @@ type selEQInt32Float64Op struct { } func (p *selEQInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5046,12 +4851,6 @@ type selEQInt32DecimalConstOp struct { } func (p *selEQInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5184,12 +4983,6 @@ type selEQInt32DecimalOp struct { } func (p *selEQInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5334,12 +5127,6 @@ type selEQInt64Int16ConstOp struct { } func (p *selEQInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5492,12 +5279,6 @@ type selEQInt64Int16Op struct { } func (p *selEQInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5662,12 +5443,6 @@ type selEQInt64Int32ConstOp struct { } func (p *selEQInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5820,12 +5595,6 @@ type selEQInt64Int32Op struct { } func (p *selEQInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -5990,12 +5759,6 @@ type selEQInt64Int64ConstOp struct { } func (p *selEQInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6148,12 +5911,6 @@ type selEQInt64Int64Op struct { } func (p *selEQInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6318,12 +6075,6 @@ type selEQInt64Float64ConstOp struct { } func (p *selEQInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6508,12 +6259,6 @@ type selEQInt64Float64Op struct { } func (p *selEQInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6710,12 +6455,6 @@ type selEQInt64DecimalConstOp struct { } func (p *selEQInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6848,12 +6587,6 @@ type selEQInt64DecimalOp struct { } func (p *selEQInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -6998,12 +6731,6 @@ type selEQFloat64Int16ConstOp struct { } func (p *selEQFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -7188,12 +6915,6 @@ type selEQFloat64Int16Op struct { } func (p *selEQFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -7390,12 +7111,6 @@ type selEQFloat64Int32ConstOp struct { } func (p *selEQFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -7580,12 +7295,6 @@ type selEQFloat64Int32Op struct { } func (p *selEQFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -7782,12 +7491,6 @@ type selEQFloat64Int64ConstOp struct { } func (p *selEQFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -7972,12 +7675,6 @@ type selEQFloat64Int64Op struct { } func (p *selEQFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -8174,12 +7871,6 @@ type selEQFloat64Float64ConstOp struct { } func (p *selEQFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -8364,12 +8055,6 @@ type selEQFloat64Float64Op struct { } func (p *selEQFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -8566,12 +8251,6 @@ type selEQFloat64DecimalConstOp struct { } func (p *selEQFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -8712,12 +8391,6 @@ type selEQFloat64DecimalOp struct { } func (p *selEQFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -8870,12 +8543,6 @@ type selEQTimestampTimestampConstOp struct { } func (p *selEQTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9012,12 +8679,6 @@ type selEQTimestampTimestampOp struct { } func (p *selEQTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9166,12 +8827,6 @@ type selEQIntervalIntervalConstOp struct { } func (p *selEQIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9280,12 +8935,6 @@ type selEQIntervalIntervalOp struct { } func (p *selEQIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9406,12 +9055,6 @@ type selEQJSONJSONConstOp struct { } func (p *selEQJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9542,12 +9185,6 @@ type selEQJSONJSONOp struct { } func (p *selEQJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9688,12 +9325,6 @@ type selEQDatumDatumConstOp struct { } func (p *selEQDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9808,12 +9439,6 @@ type selEQDatumDatumOp struct { } func (p *selEQDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -9938,12 +9563,6 @@ type selNEBoolBoolConstOp struct { } func (p *selNEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10084,12 +9703,6 @@ type selNEBoolBoolOp struct { } func (p *selNEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10242,12 +9855,6 @@ type selNEBytesBytesConstOp struct { } func (p *selNEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10354,12 +9961,6 @@ type selNEBytesBytesOp struct { } func (p *selNEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10476,12 +10077,6 @@ type selNEDecimalInt16ConstOp struct { } func (p *selNEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10614,12 +10209,6 @@ type selNEDecimalInt16Op struct { } func (p *selNEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10764,12 +10353,6 @@ type selNEDecimalInt32ConstOp struct { } func (p *selNEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -10902,12 +10485,6 @@ type selNEDecimalInt32Op struct { } func (p *selNEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11052,12 +10629,6 @@ type selNEDecimalInt64ConstOp struct { } func (p *selNEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11190,12 +10761,6 @@ type selNEDecimalInt64Op struct { } func (p *selNEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11340,12 +10905,6 @@ type selNEDecimalFloat64ConstOp struct { } func (p *selNEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11486,12 +11045,6 @@ type selNEDecimalFloat64Op struct { } func (p *selNEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11644,12 +11197,6 @@ type selNEDecimalDecimalConstOp struct { } func (p *selNEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11758,12 +11305,6 @@ type selNEDecimalDecimalOp struct { } func (p *selNEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -11884,12 +11425,6 @@ type selNEInt16Int16ConstOp struct { } func (p *selNEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12042,12 +11577,6 @@ type selNEInt16Int16Op struct { } func (p *selNEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12212,12 +11741,6 @@ type selNEInt16Int32ConstOp struct { } func (p *selNEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12370,12 +11893,6 @@ type selNEInt16Int32Op struct { } func (p *selNEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12540,12 +12057,6 @@ type selNEInt16Int64ConstOp struct { } func (p *selNEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12698,12 +12209,6 @@ type selNEInt16Int64Op struct { } func (p *selNEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -12868,12 +12373,6 @@ type selNEInt16Float64ConstOp struct { } func (p *selNEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13058,12 +12557,6 @@ type selNEInt16Float64Op struct { } func (p *selNEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13260,12 +12753,6 @@ type selNEInt16DecimalConstOp struct { } func (p *selNEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13398,12 +12885,6 @@ type selNEInt16DecimalOp struct { } func (p *selNEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13548,12 +13029,6 @@ type selNEInt32Int16ConstOp struct { } func (p *selNEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13706,12 +13181,6 @@ type selNEInt32Int16Op struct { } func (p *selNEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -13876,12 +13345,6 @@ type selNEInt32Int32ConstOp struct { } func (p *selNEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14034,12 +13497,6 @@ type selNEInt32Int32Op struct { } func (p *selNEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14204,12 +13661,6 @@ type selNEInt32Int64ConstOp struct { } func (p *selNEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14362,12 +13813,6 @@ type selNEInt32Int64Op struct { } func (p *selNEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14532,12 +13977,6 @@ type selNEInt32Float64ConstOp struct { } func (p *selNEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14722,12 +14161,6 @@ type selNEInt32Float64Op struct { } func (p *selNEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -14924,12 +14357,6 @@ type selNEInt32DecimalConstOp struct { } func (p *selNEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15062,12 +14489,6 @@ type selNEInt32DecimalOp struct { } func (p *selNEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15212,12 +14633,6 @@ type selNEInt64Int16ConstOp struct { } func (p *selNEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15370,12 +14785,6 @@ type selNEInt64Int16Op struct { } func (p *selNEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15540,12 +14949,6 @@ type selNEInt64Int32ConstOp struct { } func (p *selNEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15698,12 +15101,6 @@ type selNEInt64Int32Op struct { } func (p *selNEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -15868,12 +15265,6 @@ type selNEInt64Int64ConstOp struct { } func (p *selNEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16026,12 +15417,6 @@ type selNEInt64Int64Op struct { } func (p *selNEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16196,12 +15581,6 @@ type selNEInt64Float64ConstOp struct { } func (p *selNEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16386,12 +15765,6 @@ type selNEInt64Float64Op struct { } func (p *selNEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16588,12 +15961,6 @@ type selNEInt64DecimalConstOp struct { } func (p *selNEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16726,12 +16093,6 @@ type selNEInt64DecimalOp struct { } func (p *selNEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -16876,12 +16237,6 @@ type selNEFloat64Int16ConstOp struct { } func (p *selNEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -17066,12 +16421,6 @@ type selNEFloat64Int16Op struct { } func (p *selNEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -17268,12 +16617,6 @@ type selNEFloat64Int32ConstOp struct { } func (p *selNEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -17458,12 +16801,6 @@ type selNEFloat64Int32Op struct { } func (p *selNEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -17660,12 +16997,6 @@ type selNEFloat64Int64ConstOp struct { } func (p *selNEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -17850,12 +17181,6 @@ type selNEFloat64Int64Op struct { } func (p *selNEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18052,12 +17377,6 @@ type selNEFloat64Float64ConstOp struct { } func (p *selNEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18242,12 +17561,6 @@ type selNEFloat64Float64Op struct { } func (p *selNEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18444,12 +17757,6 @@ type selNEFloat64DecimalConstOp struct { } func (p *selNEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18590,12 +17897,6 @@ type selNEFloat64DecimalOp struct { } func (p *selNEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18748,12 +18049,6 @@ type selNETimestampTimestampConstOp struct { } func (p *selNETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -18890,12 +18185,6 @@ type selNETimestampTimestampOp struct { } func (p *selNETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19044,12 +18333,6 @@ type selNEIntervalIntervalConstOp struct { } func (p *selNEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19158,12 +18441,6 @@ type selNEIntervalIntervalOp struct { } func (p *selNEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19284,12 +18561,6 @@ type selNEJSONJSONConstOp struct { } func (p *selNEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19420,12 +18691,6 @@ type selNEJSONJSONOp struct { } func (p *selNEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19566,12 +18831,6 @@ type selNEDatumDatumConstOp struct { } func (p *selNEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19686,12 +18945,6 @@ type selNEDatumDatumOp struct { } func (p *selNEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19816,12 +19069,6 @@ type selLTBoolBoolConstOp struct { } func (p *selLTBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -19962,12 +19209,6 @@ type selLTBoolBoolOp struct { } func (p *selLTBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20120,12 +19361,6 @@ type selLTBytesBytesConstOp struct { } func (p *selLTBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20232,12 +19467,6 @@ type selLTBytesBytesOp struct { } func (p *selLTBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20354,12 +19583,6 @@ type selLTDecimalInt16ConstOp struct { } func (p *selLTDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20492,12 +19715,6 @@ type selLTDecimalInt16Op struct { } func (p *selLTDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20642,12 +19859,6 @@ type selLTDecimalInt32ConstOp struct { } func (p *selLTDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20780,12 +19991,6 @@ type selLTDecimalInt32Op struct { } func (p *selLTDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -20930,12 +20135,6 @@ type selLTDecimalInt64ConstOp struct { } func (p *selLTDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21068,12 +20267,6 @@ type selLTDecimalInt64Op struct { } func (p *selLTDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21218,12 +20411,6 @@ type selLTDecimalFloat64ConstOp struct { } func (p *selLTDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21364,12 +20551,6 @@ type selLTDecimalFloat64Op struct { } func (p *selLTDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21522,12 +20703,6 @@ type selLTDecimalDecimalConstOp struct { } func (p *selLTDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21636,12 +20811,6 @@ type selLTDecimalDecimalOp struct { } func (p *selLTDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21762,12 +20931,6 @@ type selLTInt16Int16ConstOp struct { } func (p *selLTInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -21920,12 +21083,6 @@ type selLTInt16Int16Op struct { } func (p *selLTInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22090,12 +21247,6 @@ type selLTInt16Int32ConstOp struct { } func (p *selLTInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22248,12 +21399,6 @@ type selLTInt16Int32Op struct { } func (p *selLTInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22418,12 +21563,6 @@ type selLTInt16Int64ConstOp struct { } func (p *selLTInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22576,12 +21715,6 @@ type selLTInt16Int64Op struct { } func (p *selLTInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22746,12 +21879,6 @@ type selLTInt16Float64ConstOp struct { } func (p *selLTInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -22936,12 +22063,6 @@ type selLTInt16Float64Op struct { } func (p *selLTInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23138,12 +22259,6 @@ type selLTInt16DecimalConstOp struct { } func (p *selLTInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23276,12 +22391,6 @@ type selLTInt16DecimalOp struct { } func (p *selLTInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23426,12 +22535,6 @@ type selLTInt32Int16ConstOp struct { } func (p *selLTInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23584,12 +22687,6 @@ type selLTInt32Int16Op struct { } func (p *selLTInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23754,12 +22851,6 @@ type selLTInt32Int32ConstOp struct { } func (p *selLTInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -23912,12 +23003,6 @@ type selLTInt32Int32Op struct { } func (p *selLTInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24082,12 +23167,6 @@ type selLTInt32Int64ConstOp struct { } func (p *selLTInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24240,12 +23319,6 @@ type selLTInt32Int64Op struct { } func (p *selLTInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24410,12 +23483,6 @@ type selLTInt32Float64ConstOp struct { } func (p *selLTInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24600,12 +23667,6 @@ type selLTInt32Float64Op struct { } func (p *selLTInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24802,12 +23863,6 @@ type selLTInt32DecimalConstOp struct { } func (p *selLTInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -24940,12 +23995,6 @@ type selLTInt32DecimalOp struct { } func (p *selLTInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25090,12 +24139,6 @@ type selLTInt64Int16ConstOp struct { } func (p *selLTInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25248,12 +24291,6 @@ type selLTInt64Int16Op struct { } func (p *selLTInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25418,12 +24455,6 @@ type selLTInt64Int32ConstOp struct { } func (p *selLTInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25576,12 +24607,6 @@ type selLTInt64Int32Op struct { } func (p *selLTInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25746,12 +24771,6 @@ type selLTInt64Int64ConstOp struct { } func (p *selLTInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -25904,12 +24923,6 @@ type selLTInt64Int64Op struct { } func (p *selLTInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26074,12 +25087,6 @@ type selLTInt64Float64ConstOp struct { } func (p *selLTInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26264,12 +25271,6 @@ type selLTInt64Float64Op struct { } func (p *selLTInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26466,12 +25467,6 @@ type selLTInt64DecimalConstOp struct { } func (p *selLTInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26604,12 +25599,6 @@ type selLTInt64DecimalOp struct { } func (p *selLTInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26754,12 +25743,6 @@ type selLTFloat64Int16ConstOp struct { } func (p *selLTFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -26944,12 +25927,6 @@ type selLTFloat64Int16Op struct { } func (p *selLTFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -27146,12 +26123,6 @@ type selLTFloat64Int32ConstOp struct { } func (p *selLTFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -27336,12 +26307,6 @@ type selLTFloat64Int32Op struct { } func (p *selLTFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -27538,12 +26503,6 @@ type selLTFloat64Int64ConstOp struct { } func (p *selLTFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -27728,12 +26687,6 @@ type selLTFloat64Int64Op struct { } func (p *selLTFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -27930,12 +26883,6 @@ type selLTFloat64Float64ConstOp struct { } func (p *selLTFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28120,12 +27067,6 @@ type selLTFloat64Float64Op struct { } func (p *selLTFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28322,12 +27263,6 @@ type selLTFloat64DecimalConstOp struct { } func (p *selLTFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28468,12 +27403,6 @@ type selLTFloat64DecimalOp struct { } func (p *selLTFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28626,12 +27555,6 @@ type selLTTimestampTimestampConstOp struct { } func (p *selLTTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28768,12 +27691,6 @@ type selLTTimestampTimestampOp struct { } func (p *selLTTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -28922,12 +27839,6 @@ type selLTIntervalIntervalConstOp struct { } func (p *selLTIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29036,12 +27947,6 @@ type selLTIntervalIntervalOp struct { } func (p *selLTIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29162,12 +28067,6 @@ type selLTJSONJSONConstOp struct { } func (p *selLTJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29298,12 +28197,6 @@ type selLTJSONJSONOp struct { } func (p *selLTJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29444,12 +28337,6 @@ type selLTDatumDatumConstOp struct { } func (p *selLTDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29564,12 +28451,6 @@ type selLTDatumDatumOp struct { } func (p *selLTDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29694,12 +28575,6 @@ type selLEBoolBoolConstOp struct { } func (p *selLEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29840,12 +28715,6 @@ type selLEBoolBoolOp struct { } func (p *selLEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -29998,12 +28867,6 @@ type selLEBytesBytesConstOp struct { } func (p *selLEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30110,12 +28973,6 @@ type selLEBytesBytesOp struct { } func (p *selLEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30232,12 +29089,6 @@ type selLEDecimalInt16ConstOp struct { } func (p *selLEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30370,12 +29221,6 @@ type selLEDecimalInt16Op struct { } func (p *selLEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30520,12 +29365,6 @@ type selLEDecimalInt32ConstOp struct { } func (p *selLEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30658,12 +29497,6 @@ type selLEDecimalInt32Op struct { } func (p *selLEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30808,12 +29641,6 @@ type selLEDecimalInt64ConstOp struct { } func (p *selLEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -30946,12 +29773,6 @@ type selLEDecimalInt64Op struct { } func (p *selLEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31096,12 +29917,6 @@ type selLEDecimalFloat64ConstOp struct { } func (p *selLEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31242,12 +30057,6 @@ type selLEDecimalFloat64Op struct { } func (p *selLEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31400,12 +30209,6 @@ type selLEDecimalDecimalConstOp struct { } func (p *selLEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31514,12 +30317,6 @@ type selLEDecimalDecimalOp struct { } func (p *selLEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31640,12 +30437,6 @@ type selLEInt16Int16ConstOp struct { } func (p *selLEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31798,12 +30589,6 @@ type selLEInt16Int16Op struct { } func (p *selLEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -31968,12 +30753,6 @@ type selLEInt16Int32ConstOp struct { } func (p *selLEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -32126,12 +30905,6 @@ type selLEInt16Int32Op struct { } func (p *selLEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -32296,12 +31069,6 @@ type selLEInt16Int64ConstOp struct { } func (p *selLEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -32454,12 +31221,6 @@ type selLEInt16Int64Op struct { } func (p *selLEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -32624,12 +31385,6 @@ type selLEInt16Float64ConstOp struct { } func (p *selLEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -32814,12 +31569,6 @@ type selLEInt16Float64Op struct { } func (p *selLEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33016,12 +31765,6 @@ type selLEInt16DecimalConstOp struct { } func (p *selLEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33154,12 +31897,6 @@ type selLEInt16DecimalOp struct { } func (p *selLEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33304,12 +32041,6 @@ type selLEInt32Int16ConstOp struct { } func (p *selLEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33462,12 +32193,6 @@ type selLEInt32Int16Op struct { } func (p *selLEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33632,12 +32357,6 @@ type selLEInt32Int32ConstOp struct { } func (p *selLEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33790,12 +32509,6 @@ type selLEInt32Int32Op struct { } func (p *selLEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -33960,12 +32673,6 @@ type selLEInt32Int64ConstOp struct { } func (p *selLEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34118,12 +32825,6 @@ type selLEInt32Int64Op struct { } func (p *selLEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34288,12 +32989,6 @@ type selLEInt32Float64ConstOp struct { } func (p *selLEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34478,12 +33173,6 @@ type selLEInt32Float64Op struct { } func (p *selLEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34680,12 +33369,6 @@ type selLEInt32DecimalConstOp struct { } func (p *selLEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34818,12 +33501,6 @@ type selLEInt32DecimalOp struct { } func (p *selLEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -34968,12 +33645,6 @@ type selLEInt64Int16ConstOp struct { } func (p *selLEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35126,12 +33797,6 @@ type selLEInt64Int16Op struct { } func (p *selLEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35296,12 +33961,6 @@ type selLEInt64Int32ConstOp struct { } func (p *selLEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35454,12 +34113,6 @@ type selLEInt64Int32Op struct { } func (p *selLEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35624,12 +34277,6 @@ type selLEInt64Int64ConstOp struct { } func (p *selLEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35782,12 +34429,6 @@ type selLEInt64Int64Op struct { } func (p *selLEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -35952,12 +34593,6 @@ type selLEInt64Float64ConstOp struct { } func (p *selLEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -36142,12 +34777,6 @@ type selLEInt64Float64Op struct { } func (p *selLEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -36344,12 +34973,6 @@ type selLEInt64DecimalConstOp struct { } func (p *selLEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -36482,12 +35105,6 @@ type selLEInt64DecimalOp struct { } func (p *selLEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -36632,12 +35249,6 @@ type selLEFloat64Int16ConstOp struct { } func (p *selLEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -36822,12 +35433,6 @@ type selLEFloat64Int16Op struct { } func (p *selLEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37024,12 +35629,6 @@ type selLEFloat64Int32ConstOp struct { } func (p *selLEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37214,12 +35813,6 @@ type selLEFloat64Int32Op struct { } func (p *selLEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37416,12 +36009,6 @@ type selLEFloat64Int64ConstOp struct { } func (p *selLEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37606,12 +36193,6 @@ type selLEFloat64Int64Op struct { } func (p *selLEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37808,12 +36389,6 @@ type selLEFloat64Float64ConstOp struct { } func (p *selLEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -37998,12 +36573,6 @@ type selLEFloat64Float64Op struct { } func (p *selLEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38200,12 +36769,6 @@ type selLEFloat64DecimalConstOp struct { } func (p *selLEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38346,12 +36909,6 @@ type selLEFloat64DecimalOp struct { } func (p *selLEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38504,12 +37061,6 @@ type selLETimestampTimestampConstOp struct { } func (p *selLETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38646,12 +37197,6 @@ type selLETimestampTimestampOp struct { } func (p *selLETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38800,12 +37345,6 @@ type selLEIntervalIntervalConstOp struct { } func (p *selLEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -38914,12 +37453,6 @@ type selLEIntervalIntervalOp struct { } func (p *selLEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39040,12 +37573,6 @@ type selLEJSONJSONConstOp struct { } func (p *selLEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39176,12 +37703,6 @@ type selLEJSONJSONOp struct { } func (p *selLEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39322,12 +37843,6 @@ type selLEDatumDatumConstOp struct { } func (p *selLEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39442,12 +37957,6 @@ type selLEDatumDatumOp struct { } func (p *selLEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39572,12 +38081,6 @@ type selGTBoolBoolConstOp struct { } func (p *selGTBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39718,12 +38221,6 @@ type selGTBoolBoolOp struct { } func (p *selGTBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39876,12 +38373,6 @@ type selGTBytesBytesConstOp struct { } func (p *selGTBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -39988,12 +38479,6 @@ type selGTBytesBytesOp struct { } func (p *selGTBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40110,12 +38595,6 @@ type selGTDecimalInt16ConstOp struct { } func (p *selGTDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40248,12 +38727,6 @@ type selGTDecimalInt16Op struct { } func (p *selGTDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40398,12 +38871,6 @@ type selGTDecimalInt32ConstOp struct { } func (p *selGTDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40536,12 +39003,6 @@ type selGTDecimalInt32Op struct { } func (p *selGTDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40686,12 +39147,6 @@ type selGTDecimalInt64ConstOp struct { } func (p *selGTDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40824,12 +39279,6 @@ type selGTDecimalInt64Op struct { } func (p *selGTDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -40974,12 +39423,6 @@ type selGTDecimalFloat64ConstOp struct { } func (p *selGTDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41120,12 +39563,6 @@ type selGTDecimalFloat64Op struct { } func (p *selGTDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41278,12 +39715,6 @@ type selGTDecimalDecimalConstOp struct { } func (p *selGTDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41392,12 +39823,6 @@ type selGTDecimalDecimalOp struct { } func (p *selGTDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41518,12 +39943,6 @@ type selGTInt16Int16ConstOp struct { } func (p *selGTInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41676,12 +40095,6 @@ type selGTInt16Int16Op struct { } func (p *selGTInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -41846,12 +40259,6 @@ type selGTInt16Int32ConstOp struct { } func (p *selGTInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42004,12 +40411,6 @@ type selGTInt16Int32Op struct { } func (p *selGTInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42174,12 +40575,6 @@ type selGTInt16Int64ConstOp struct { } func (p *selGTInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42332,12 +40727,6 @@ type selGTInt16Int64Op struct { } func (p *selGTInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42502,12 +40891,6 @@ type selGTInt16Float64ConstOp struct { } func (p *selGTInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42692,12 +41075,6 @@ type selGTInt16Float64Op struct { } func (p *selGTInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -42894,12 +41271,6 @@ type selGTInt16DecimalConstOp struct { } func (p *selGTInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43032,12 +41403,6 @@ type selGTInt16DecimalOp struct { } func (p *selGTInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43182,12 +41547,6 @@ type selGTInt32Int16ConstOp struct { } func (p *selGTInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43340,12 +41699,6 @@ type selGTInt32Int16Op struct { } func (p *selGTInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43510,12 +41863,6 @@ type selGTInt32Int32ConstOp struct { } func (p *selGTInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43668,12 +42015,6 @@ type selGTInt32Int32Op struct { } func (p *selGTInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43838,12 +42179,6 @@ type selGTInt32Int64ConstOp struct { } func (p *selGTInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -43996,12 +42331,6 @@ type selGTInt32Int64Op struct { } func (p *selGTInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -44166,12 +42495,6 @@ type selGTInt32Float64ConstOp struct { } func (p *selGTInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -44356,12 +42679,6 @@ type selGTInt32Float64Op struct { } func (p *selGTInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -44558,12 +42875,6 @@ type selGTInt32DecimalConstOp struct { } func (p *selGTInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -44696,12 +43007,6 @@ type selGTInt32DecimalOp struct { } func (p *selGTInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -44846,12 +43151,6 @@ type selGTInt64Int16ConstOp struct { } func (p *selGTInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45004,12 +43303,6 @@ type selGTInt64Int16Op struct { } func (p *selGTInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45174,12 +43467,6 @@ type selGTInt64Int32ConstOp struct { } func (p *selGTInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45332,12 +43619,6 @@ type selGTInt64Int32Op struct { } func (p *selGTInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45502,12 +43783,6 @@ type selGTInt64Int64ConstOp struct { } func (p *selGTInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45660,12 +43935,6 @@ type selGTInt64Int64Op struct { } func (p *selGTInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -45830,12 +44099,6 @@ type selGTInt64Float64ConstOp struct { } func (p *selGTInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46020,12 +44283,6 @@ type selGTInt64Float64Op struct { } func (p *selGTInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46222,12 +44479,6 @@ type selGTInt64DecimalConstOp struct { } func (p *selGTInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46360,12 +44611,6 @@ type selGTInt64DecimalOp struct { } func (p *selGTInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46510,12 +44755,6 @@ type selGTFloat64Int16ConstOp struct { } func (p *selGTFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46700,12 +44939,6 @@ type selGTFloat64Int16Op struct { } func (p *selGTFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -46902,12 +45135,6 @@ type selGTFloat64Int32ConstOp struct { } func (p *selGTFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -47092,12 +45319,6 @@ type selGTFloat64Int32Op struct { } func (p *selGTFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -47294,12 +45515,6 @@ type selGTFloat64Int64ConstOp struct { } func (p *selGTFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -47484,12 +45699,6 @@ type selGTFloat64Int64Op struct { } func (p *selGTFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -47686,12 +45895,6 @@ type selGTFloat64Float64ConstOp struct { } func (p *selGTFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -47876,12 +46079,6 @@ type selGTFloat64Float64Op struct { } func (p *selGTFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48078,12 +46275,6 @@ type selGTFloat64DecimalConstOp struct { } func (p *selGTFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48224,12 +46415,6 @@ type selGTFloat64DecimalOp struct { } func (p *selGTFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48382,12 +46567,6 @@ type selGTTimestampTimestampConstOp struct { } func (p *selGTTimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48524,12 +46703,6 @@ type selGTTimestampTimestampOp struct { } func (p *selGTTimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48678,12 +46851,6 @@ type selGTIntervalIntervalConstOp struct { } func (p *selGTIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48792,12 +46959,6 @@ type selGTIntervalIntervalOp struct { } func (p *selGTIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -48918,12 +47079,6 @@ type selGTJSONJSONConstOp struct { } func (p *selGTJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49054,12 +47209,6 @@ type selGTJSONJSONOp struct { } func (p *selGTJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49200,12 +47349,6 @@ type selGTDatumDatumConstOp struct { } func (p *selGTDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49320,12 +47463,6 @@ type selGTDatumDatumOp struct { } func (p *selGTDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49450,12 +47587,6 @@ type selGEBoolBoolConstOp struct { } func (p *selGEBoolBoolConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49596,12 +47727,6 @@ type selGEBoolBoolOp struct { } func (p *selGEBoolBoolOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49754,12 +47879,6 @@ type selGEBytesBytesConstOp struct { } func (p *selGEBytesBytesConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49866,12 +47985,6 @@ type selGEBytesBytesOp struct { } func (p *selGEBytesBytesOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -49988,12 +48101,6 @@ type selGEDecimalInt16ConstOp struct { } func (p *selGEDecimalInt16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50126,12 +48233,6 @@ type selGEDecimalInt16Op struct { } func (p *selGEDecimalInt16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50276,12 +48377,6 @@ type selGEDecimalInt32ConstOp struct { } func (p *selGEDecimalInt32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50414,12 +48509,6 @@ type selGEDecimalInt32Op struct { } func (p *selGEDecimalInt32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50564,12 +48653,6 @@ type selGEDecimalInt64ConstOp struct { } func (p *selGEDecimalInt64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50702,12 +48785,6 @@ type selGEDecimalInt64Op struct { } func (p *selGEDecimalInt64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50852,12 +48929,6 @@ type selGEDecimalFloat64ConstOp struct { } func (p *selGEDecimalFloat64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -50998,12 +49069,6 @@ type selGEDecimalFloat64Op struct { } func (p *selGEDecimalFloat64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51156,12 +49221,6 @@ type selGEDecimalDecimalConstOp struct { } func (p *selGEDecimalDecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51270,12 +49329,6 @@ type selGEDecimalDecimalOp struct { } func (p *selGEDecimalDecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51396,12 +49449,6 @@ type selGEInt16Int16ConstOp struct { } func (p *selGEInt16Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51554,12 +49601,6 @@ type selGEInt16Int16Op struct { } func (p *selGEInt16Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51724,12 +49765,6 @@ type selGEInt16Int32ConstOp struct { } func (p *selGEInt16Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -51882,12 +49917,6 @@ type selGEInt16Int32Op struct { } func (p *selGEInt16Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52052,12 +50081,6 @@ type selGEInt16Int64ConstOp struct { } func (p *selGEInt16Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52210,12 +50233,6 @@ type selGEInt16Int64Op struct { } func (p *selGEInt16Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52380,12 +50397,6 @@ type selGEInt16Float64ConstOp struct { } func (p *selGEInt16Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52570,12 +50581,6 @@ type selGEInt16Float64Op struct { } func (p *selGEInt16Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52772,12 +50777,6 @@ type selGEInt16DecimalConstOp struct { } func (p *selGEInt16DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -52910,12 +50909,6 @@ type selGEInt16DecimalOp struct { } func (p *selGEInt16DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53060,12 +51053,6 @@ type selGEInt32Int16ConstOp struct { } func (p *selGEInt32Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53218,12 +51205,6 @@ type selGEInt32Int16Op struct { } func (p *selGEInt32Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53388,12 +51369,6 @@ type selGEInt32Int32ConstOp struct { } func (p *selGEInt32Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53546,12 +51521,6 @@ type selGEInt32Int32Op struct { } func (p *selGEInt32Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53716,12 +51685,6 @@ type selGEInt32Int64ConstOp struct { } func (p *selGEInt32Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -53874,12 +51837,6 @@ type selGEInt32Int64Op struct { } func (p *selGEInt32Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54044,12 +52001,6 @@ type selGEInt32Float64ConstOp struct { } func (p *selGEInt32Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54234,12 +52185,6 @@ type selGEInt32Float64Op struct { } func (p *selGEInt32Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54436,12 +52381,6 @@ type selGEInt32DecimalConstOp struct { } func (p *selGEInt32DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54574,12 +52513,6 @@ type selGEInt32DecimalOp struct { } func (p *selGEInt32DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54724,12 +52657,6 @@ type selGEInt64Int16ConstOp struct { } func (p *selGEInt64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -54882,12 +52809,6 @@ type selGEInt64Int16Op struct { } func (p *selGEInt64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55052,12 +52973,6 @@ type selGEInt64Int32ConstOp struct { } func (p *selGEInt64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55210,12 +53125,6 @@ type selGEInt64Int32Op struct { } func (p *selGEInt64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55380,12 +53289,6 @@ type selGEInt64Int64ConstOp struct { } func (p *selGEInt64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55538,12 +53441,6 @@ type selGEInt64Int64Op struct { } func (p *selGEInt64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55708,12 +53605,6 @@ type selGEInt64Float64ConstOp struct { } func (p *selGEInt64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -55898,12 +53789,6 @@ type selGEInt64Float64Op struct { } func (p *selGEInt64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56100,12 +53985,6 @@ type selGEInt64DecimalConstOp struct { } func (p *selGEInt64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56238,12 +54117,6 @@ type selGEInt64DecimalOp struct { } func (p *selGEInt64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56388,12 +54261,6 @@ type selGEFloat64Int16ConstOp struct { } func (p *selGEFloat64Int16ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56578,12 +54445,6 @@ type selGEFloat64Int16Op struct { } func (p *selGEFloat64Int16Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56780,12 +54641,6 @@ type selGEFloat64Int32ConstOp struct { } func (p *selGEFloat64Int32ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -56970,12 +54825,6 @@ type selGEFloat64Int32Op struct { } func (p *selGEFloat64Int32Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -57172,12 +55021,6 @@ type selGEFloat64Int64ConstOp struct { } func (p *selGEFloat64Int64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -57362,12 +55205,6 @@ type selGEFloat64Int64Op struct { } func (p *selGEFloat64Int64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -57564,12 +55401,6 @@ type selGEFloat64Float64ConstOp struct { } func (p *selGEFloat64Float64ConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -57754,12 +55585,6 @@ type selGEFloat64Float64Op struct { } func (p *selGEFloat64Float64Op) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -57956,12 +55781,6 @@ type selGEFloat64DecimalConstOp struct { } func (p *selGEFloat64DecimalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58102,12 +55921,6 @@ type selGEFloat64DecimalOp struct { } func (p *selGEFloat64DecimalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58260,12 +56073,6 @@ type selGETimestampTimestampConstOp struct { } func (p *selGETimestampTimestampConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58402,12 +56209,6 @@ type selGETimestampTimestampOp struct { } func (p *selGETimestampTimestampOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58556,12 +56357,6 @@ type selGEIntervalIntervalConstOp struct { } func (p *selGEIntervalIntervalConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58670,12 +56465,6 @@ type selGEIntervalIntervalOp struct { } func (p *selGEIntervalIntervalOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58796,12 +56585,6 @@ type selGEJSONJSONConstOp struct { } func (p *selGEJSONJSONConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -58932,12 +56715,6 @@ type selGEJSONJSONOp struct { } func (p *selGEJSONJSONOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -59078,12 +56855,6 @@ type selGEDatumDatumConstOp struct { } func (p *selGEDatumDatumConstOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -59198,12 +56969,6 @@ type selGEDatumDatumOp struct { } func (p *selGEDatumDatumOp) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { diff --git a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go index e5c31c139175..c0a4c4922c17 100644 --- a/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go +++ b/pkg/sql/colexec/colexecsel/selection_ops_tmpl.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/sql/colconv" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen" "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" "github.com/cockroachdb/cockroach/pkg/sql/colexecop" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -176,16 +175,14 @@ func _SEL_LOOP(_HAS_NULLS bool) { // */}} // constant, except for the constant itself. type selConstOpBase struct { colexecop.OneInputHelper - colIdx int - overloadHelper execgen.OverloadHelper + colIdx int } // selOpBase contains all of the fields for non-constant binary selections. type selOpBase struct { colexecop.OneInputHelper - col1Idx int - col2Idx int - overloadHelper execgen.OverloadHelper + col1Idx int + col2Idx int } // {{define "selConstOp"}} @@ -195,12 +192,6 @@ type _OP_CONST_NAME struct { } func (p *_OP_CONST_NAME) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { @@ -232,12 +223,6 @@ type _OP_NAME struct { } func (p *_OP_NAME) Next() coldata.Batch { - // In order to inline the templated code of overloads, we need to have a - // `_overloadHelper` local variable of type `execgen.OverloadHelper`. - _overloadHelper := p.overloadHelper - // However, the scratch is not used in all of the selection operators, so - // we add this to go around "unused" error. - _ = _overloadHelper for { batch := p.Input.Next() if batch.Length() == 0 { diff --git a/pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go b/pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go index 0ace6d83e271..c2be2770cf7a 100644 --- a/pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go +++ b/pkg/sql/colexec/colexecwindow/range_offset_handler.eg.go @@ -76,26 +76,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetPrecedingStartAscInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetPrecedingStartAscInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -105,9 +96,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -117,9 +105,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -129,9 +114,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -141,9 +123,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -153,9 +132,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartAscTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -167,7 +143,7 @@ func newRangeOffsetHandler( } _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -179,26 +155,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetPrecedingStartDescInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetPrecedingStartDescInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -208,9 +175,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -220,9 +184,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -232,9 +193,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -244,9 +202,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -256,9 +211,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingStartDescTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -270,7 +222,7 @@ func newRangeOffsetHandler( } binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -285,26 +237,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetPrecedingEndAscInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetPrecedingEndAscInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -314,9 +257,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -326,9 +266,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -338,9 +275,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -350,9 +284,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -362,9 +293,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndAscTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -376,7 +304,7 @@ func newRangeOffsetHandler( } _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -388,26 +316,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetPrecedingEndDescInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetPrecedingEndDescInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -417,9 +336,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -429,9 +345,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -441,9 +354,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -453,9 +363,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -465,9 +372,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetPrecedingEndDescTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -479,7 +383,7 @@ func newRangeOffsetHandler( } binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -497,26 +401,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetFollowingStartAscInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetFollowingStartAscInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -526,9 +421,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -538,9 +430,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -550,9 +439,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -562,9 +448,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -574,9 +457,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartAscTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -588,7 +468,7 @@ func newRangeOffsetHandler( } binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -600,26 +480,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetFollowingStartDescInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetFollowingStartDescInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -629,9 +500,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -641,9 +509,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -653,9 +518,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -665,9 +527,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -677,9 +536,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingStartDescTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -691,7 +547,7 @@ func newRangeOffsetHandler( } _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -706,26 +562,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetFollowingEndAscInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetFollowingEndAscInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -735,9 +582,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -747,9 +591,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -759,9 +600,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -771,9 +609,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -783,9 +618,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndAscTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -797,7 +629,7 @@ func newRangeOffsetHandler( } binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -809,26 +641,17 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescInt16{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int16), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case 32: op := &rangeHandlerOffsetFollowingEndDescInt32{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int32), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op case -1: default: op := &rangeHandlerOffsetFollowingEndDescInt64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(int64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DecimalFamily: @@ -838,9 +661,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescDecimal{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(apd.Decimal), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.FloatFamily: @@ -850,9 +670,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescFloat64{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(float64), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.IntervalFamily: @@ -862,9 +679,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescInterval{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.DateFamily: @@ -874,9 +688,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescDate{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimestampTZFamily, types.TimestampFamily: @@ -886,9 +697,6 @@ func newRangeOffsetHandler( op := &rangeHandlerOffsetFollowingEndDescTimestamp{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(duration.Duration), } - _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( - ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } case types.TimeTZFamily, types.TimeFamily: @@ -900,7 +708,7 @@ func newRangeOffsetHandler( } _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} return op } } @@ -914,10 +722,9 @@ func newRangeOffsetHandler( // rangeOffsetHandlerBase extracts common fields and methods of the // rangeOffsetHandler utility operators. type rangeOffsetHandlerBase struct { - storedCols *colexecutils.SpillingBuffer - ordColIdx int - peersColIdx int - overloadHelper execgen.OverloadHelper + storedCols *colexecutils.SpillingBuffer + ordColIdx int + peersColIdx int } // rangeHandlerOffsetPrecedingStartAscInt16 is a utility operator used to retrieve the location of @@ -948,11 +755,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1097,11 +899,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1246,11 +1043,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1395,11 +1187,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1533,11 +1320,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1687,11 +1469,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1818,11 +1595,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -1979,11 +1751,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2098,7 +1865,8 @@ func (h *rangeHandlerOffsetPrecedingStartAscTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetPrecedingStartAscDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscDatum{} @@ -2122,11 +1890,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartAscDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartAscDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2260,11 +2028,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2382,11 +2145,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2504,11 +2262,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2626,11 +2379,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2737,11 +2485,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2864,11 +2607,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -2968,11 +2706,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3102,11 +2835,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3194,7 +2922,8 @@ func (h *rangeHandlerOffsetPrecedingStartDescTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetPrecedingStartDescDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescDatum{} @@ -3218,11 +2947,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingStartDescDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingStartDescDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3329,11 +3058,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3495,11 +3219,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3661,11 +3380,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3827,11 +3541,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -3982,11 +3691,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4153,11 +3857,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4301,11 +4000,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4479,11 +4173,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4615,7 +4304,8 @@ func (h *rangeHandlerOffsetPrecedingEndAscTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetPrecedingEndAscDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscDatum{} @@ -4639,11 +4329,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndAscDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndAscDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4794,11 +4484,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -4933,11 +4618,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5072,11 +4752,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5211,11 +4886,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5339,11 +5009,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5483,11 +5148,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5604,11 +5264,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5755,11 +5410,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -5864,7 +5514,8 @@ func (h *rangeHandlerOffsetPrecedingEndDescTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetPrecedingEndDescDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescDatum{} @@ -5888,11 +5539,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetPrecedingEndDescDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetPrecedingEndDescDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6016,11 +5667,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6165,11 +5811,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6314,11 +5955,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6463,11 +6099,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6601,11 +6232,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6755,11 +6381,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -6886,11 +6507,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7047,11 +6663,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7166,7 +6777,8 @@ func (h *rangeHandlerOffsetFollowingStartAscTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetFollowingStartAscDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscDatum{} @@ -7190,11 +6802,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartAscDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartAscDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7328,11 +6940,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7450,11 +7057,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7572,11 +7174,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7694,11 +7291,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7805,11 +7397,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -7932,11 +7519,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8036,11 +7618,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8170,11 +7747,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8262,7 +7834,8 @@ func (h *rangeHandlerOffsetFollowingStartDescTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetFollowingStartDescDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescDatum{} @@ -8286,11 +7859,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingStartDescDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingStartDescDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8397,11 +7970,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8563,11 +8131,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8729,11 +8292,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -8895,11 +8453,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9050,11 +8603,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9221,11 +8769,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9369,11 +8912,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9547,11 +9085,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9683,7 +9216,8 @@ func (h *rangeHandlerOffsetFollowingEndAscTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetFollowingEndAscDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscDatum{} @@ -9707,11 +9241,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndAscDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndAscDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -9862,11 +9396,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescInt16{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescInt16) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10001,11 +9530,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescInt32{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescInt32) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10140,11 +9664,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescInt64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescInt64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10279,11 +9798,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescDecimal{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescDecimal) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10407,11 +9921,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescFloat64{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescFloat64) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10551,11 +10060,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescInterval{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescInterval) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10672,11 +10176,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescDate{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescDate) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10823,11 +10322,6 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescTimestamp{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescTimestamp) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. - _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx @@ -10932,7 +10426,8 @@ func (h *rangeHandlerOffsetFollowingEndDescTimestamp) close() { // the start or end bound for each row when in RANGE mode with an offset. type rangeHandlerOffsetFollowingEndDescDatum struct { rangeOffsetHandlerBase - offset tree.Datum + overloadHelper execgen.BinaryOverloadHelper + offset tree.Datum } var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescDatum{} @@ -10956,11 +10451,11 @@ var _ rangeOffsetHandler = &rangeHandlerOffsetFollowingEndDescDatum{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *rangeHandlerOffsetFollowingEndDescDatum) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. if lastIdx >= h.storedCols.Length() { return lastIdx diff --git a/pkg/sql/colexec/colexecwindow/range_offset_handler_tmpl.go b/pkg/sql/colexec/colexecwindow/range_offset_handler_tmpl.go index 86ae85cafd82..1d8d31ba287c 100644 --- a/pkg/sql/colexec/colexecwindow/range_offset_handler_tmpl.go +++ b/pkg/sql/colexec/colexecwindow/range_offset_handler_tmpl.go @@ -118,6 +118,7 @@ func newRangeOffsetHandler( op := &_OP_STRING{ offset: decodeOffset(datumAlloc, ordColType, bound.TypedOffset).(_OFFSET_GOTYPE), } + // {{if eq .VecMethod "Datum"}} // {{if .BinOpIsPlus}} binOp, _, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) @@ -125,7 +126,8 @@ func newRangeOffsetHandler( _, binOp, _ := tree.WindowFrameRangeOps{}.LookupImpl( ordColType, getOffsetType(ordColType)) // {{end}} - op.overloadHelper = execgen.OverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + op.overloadHelper = execgen.BinaryOverloadHelper{BinFn: binOp.Fn, EvalCtx: evalCtx} + // {{end}} return op // {{end}} } @@ -144,10 +146,9 @@ func newRangeOffsetHandler( // rangeOffsetHandlerBase extracts common fields and methods of the // rangeOffsetHandler utility operators. type rangeOffsetHandlerBase struct { - storedCols *colexecutils.SpillingBuffer - ordColIdx int - peersColIdx int - overloadHelper execgen.OverloadHelper + storedCols *colexecutils.SpillingBuffer + ordColIdx int + peersColIdx int } // {{range .}} @@ -160,6 +161,9 @@ type rangeOffsetHandlerBase struct { // the start or end bound for each row when in RANGE mode with an offset. type _OP_STRING struct { rangeOffsetHandlerBase + // {{if eq .VecMethod "Datum"}} + overloadHelper execgen.BinaryOverloadHelper + // {{end}} offset _OFFSET_GOTYPE } @@ -187,11 +191,13 @@ var _ rangeOffsetHandler = &_OP_STRING{} // the partition, whichever comes first. In this case, the returned index would // be '4' to indicate that the end index is the end of the partition. func (h *_OP_STRING) getIdx(ctx context.Context, currRow, lastIdx int) (idx int) { - // In order to inline the templated code of overloads, we need to have a - // "_overloadHelper" local variable of type "overloadHelper". This is - // necessary when dealing with Datum columns. + // {{if eq .VecMethod "Datum"}} + // In order to inline the templated code of the binary overloads operating + // on datums, we need to have a `_overloadHelper` local variable of type + // `execgen.BinaryOverloadHelper`. This is necessary when dealing with Time + // and TimeTZ columns since they aren't yet handled natively. _overloadHelper := h.overloadHelper - _ = _overloadHelper // Avoid unused variable warnings. + // {{end}} if lastIdx >= h.storedCols.Length() { return lastIdx diff --git a/pkg/sql/colexec/execgen/BUILD.bazel b/pkg/sql/colexec/execgen/BUILD.bazel index cd4c66c5cdcf..987042184fc9 100644 --- a/pkg/sql/colexec/execgen/BUILD.bazel +++ b/pkg/sql/colexec/execgen/BUILD.bazel @@ -5,7 +5,7 @@ go_library( srcs = [ "execgen.go", "inline.go", - "overloads_util.go", + "overloads_bin_util.go", "placeholders.go", "supported_bin_cmp_ops.go", "template.go", diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go index a2582032aad3..98b297977539 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_base.go @@ -305,7 +305,7 @@ func (o *oneArgOverload) String() string { } // twoArgsResolvedOverload is a utility struct that represents an overload that -// takes it two arguments and that has been "resolved" (meaning it supports +// takes in two arguments and that has been "resolved" (meaning it supports // only a single type family and a single type width on both sides). type twoArgsResolvedOverload struct { *overloadBase @@ -313,6 +313,12 @@ type twoArgsResolvedOverload struct { Right *lastArgWidthOverload } +// NeedsBinaryOverloadHelper returns true iff the overload is such that it needs +// access to execgen.BinaryOverloadHelper. +func (o *twoArgsResolvedOverload) NeedsBinaryOverloadHelper() bool { + return o.kind == binaryOverload && o.Right.RetVecMethod == "Datum" +} + // twoArgsResolvedOverloadsInfo contains all overloads that take in two // arguments and stores them in a similar hierarchical structure to how // twoArgsOverloads are stored, with the difference that on the "bottom" level diff --git a/pkg/sql/colexec/execgen/overloads_util.go b/pkg/sql/colexec/execgen/overloads_bin_util.go similarity index 73% rename from pkg/sql/colexec/execgen/overloads_util.go rename to pkg/sql/colexec/execgen/overloads_bin_util.go index a5c4864cb2a7..bec5a1ef73be 100644 --- a/pkg/sql/colexec/execgen/overloads_util.go +++ b/pkg/sql/colexec/execgen/overloads_bin_util.go @@ -1,4 +1,4 @@ -// Copyright 2020 The Cockroach Authors. +// Copyright 2022 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -12,13 +12,13 @@ package execgen import "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" -// OverloadHelper is a utility struct used for templates that plumbs through -// miscellaneous useful information. +// BinaryOverloadHelper is a utility struct used for templates of the binary +// overloads that fall back to the row-based tree.Datum computation. // // In order for the templates to see it correctly, a local variable named // `_overloadHelper` of this type must be declared before the inlined // overloaded code. -type OverloadHelper struct { +type BinaryOverloadHelper struct { BinFn tree.TwoArgFn EvalCtx *tree.EvalContext } diff --git a/pkg/sql/colfetcher/BUILD.bazel b/pkg/sql/colfetcher/BUILD.bazel index c30642f3f2e5..94ddbc62ab7f 100644 --- a/pkg/sql/colfetcher/BUILD.bazel +++ b/pkg/sql/colfetcher/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/col/typeconv", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvstreamer", "//pkg/roachpb:with-mocks", "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 4570ad55a3dc..17d3de0fb219 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -579,6 +580,15 @@ func getColumnTypesFromCols(cols []catalog.Column, outTypes []*types.T) []*types return outTypes } +//gcassert:inline +func (rf *cFetcher) setFetcher(f *row.KVFetcher, limitHint rowinfra.RowLimit) { + rf.fetcher = f + rf.machine.lastRowPrefix = nil + rf.machine.limitHint = int(limitHint) + rf.machine.state[0] = stateResetBatch + rf.machine.state[1] = stateInitFetch +} + // StartScan initializes and starts the key-value scan. Can be used multiple // times. // @@ -646,11 +656,30 @@ func (rf *cFetcher) StartScan( if err != nil { return err } - rf.fetcher = f - rf.machine.lastRowPrefix = nil - rf.machine.limitHint = int(limitHint) - rf.machine.state[0] = stateResetBatch - rf.machine.state[1] = stateInitFetch + rf.setFetcher(f, limitHint) + return nil +} + +// StartScanStreaming initializes and starts the key-value scan using the +// Streamer API. Can be used multiple times. +// +// The fetcher takes ownership of the spans slice - it can modify the slice and +// will perform the memory accounting accordingly. The caller can only reuse the +// spans slice after the fetcher has been closed (which happens when the fetcher +// emits the first zero batch), and if the caller does, it becomes responsible +// for the memory accounting. +func (rf *cFetcher) StartScanStreaming( + ctx context.Context, + streamer *kvstreamer.Streamer, + spans roachpb.Spans, + limitHint rowinfra.RowLimit, +) error { + kvBatchFetcher, err := row.NewTxnKVStreamer(ctx, streamer, spans, rf.lockStrength) + if err != nil { + return err + } + f := row.NewKVStreamingFetcher(kvBatchFetcher) + rf.setFetcher(f, limitHint) return nil } diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index 09f08048281e..b97d61858267 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -12,11 +12,13 @@ package colfetcher import ( "context" + "math" "sort" "time" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecargs" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan" @@ -26,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -102,6 +105,15 @@ type ColIndexJoin struct { // maintainOrdering is true when the index join is required to maintain its // input ordering, in which case the ordering of the spans cannot be changed. maintainOrdering bool + + // usesStreamer indicates whether the ColIndexJoin is using the Streamer + // API. + usesStreamer bool + streamerInfo struct { + *kvstreamer.Streamer + budgetAcc *mon.BoundAccount + budgetLimit int64 + } } var _ colexecop.KVReader = &ColIndexJoin{} @@ -119,6 +131,21 @@ func (s *ColIndexJoin) Init(ctx context.Context) { // tracing is enabled. s.Ctx, s.tracingSpan = execinfra.ProcessorSpan(s.Ctx, "colindexjoin") s.Input.Init(s.Ctx) + if s.usesStreamer { + s.streamerInfo.Streamer = kvstreamer.NewStreamer( + s.flowCtx.Cfg.DistSender, + s.flowCtx.Stopper(), + s.flowCtx.Txn, + s.flowCtx.EvalCtx.Settings, + row.GetWaitPolicy(s.rf.lockWaitPolicy), + s.streamerInfo.budgetLimit, + s.streamerInfo.budgetAcc, + ) + s.streamerInfo.Streamer.Init( + kvstreamer.OutOfOrder, + kvstreamer.Hints{UniqueRequests: true}, + ) + } } type indexJoinState uint8 @@ -174,16 +201,27 @@ func (s *ColIndexJoin) Next() coldata.Batch { // the memory accounting - we don't double count for any memory of // spans because the spanAssembler released all of the relevant // memory from its account in GetSpans(). - if err := s.rf.StartScan( - s.Ctx, - s.flowCtx.Txn, - spans, - nil, /* bsHeader */ - false, /* limitBatches */ - rowinfra.NoBytesLimit, - rowinfra.NoRowLimit, - s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes, - ); err != nil { + var err error + if s.usesStreamer { + err = s.rf.StartScanStreaming( + s.Ctx, + s.streamerInfo.Streamer, + spans, + rowinfra.NoRowLimit, + ) + } else { + err = s.rf.StartScan( + s.Ctx, + s.flowCtx.Txn, + spans, + nil, /* bsHeader */ + false, /* limitBatches */ + rowinfra.NoBytesLimit, + rowinfra.NoRowLimit, + s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes, + ) + } + if err != nil { colexecerror.InternalError(err) } s.state = indexJoinScanning @@ -385,6 +423,7 @@ func NewColIndexJoin( allocator *colmem.Allocator, fetcherAllocator *colmem.Allocator, kvFetcherMemAcc *mon.BoundAccount, + streamerBudgetAcc *mon.BoundAccount, flowCtx *execinfra.FlowCtx, helper *colexecargs.ExprHelper, input colexecop.Operator, @@ -416,19 +455,44 @@ func NewColIndexJoin( return nil, err } + memoryLimit := execinfra.GetWorkMemLimit(flowCtx) + + useStreamer := row.CanUseStreamer(ctx, flowCtx.EvalCtx.Settings) && !spec.MaintainOrdering + if useStreamer { + // TODO(yuzefovich): remove this conditional once multiple column + // families are supported. + if maxKeysPerRow, err := tableArgs.desc.KeysPerRow(tableArgs.index.GetID()); err != nil { + return nil, err + } else if maxKeysPerRow > 1 { + // Currently, the streamer only supports cases with a single column + // family. + useStreamer = false + } else { + if streamerBudgetAcc == nil { + return nil, errors.AssertionFailedf("streamer budget account is nil when the Streamer API is desired") + } + // Keep the quarter of the memory limit for the output batch of the + // cFetcher, and we'll give the remaining three quarters to the + // streamer budget below. + memoryLimit = int64(math.Ceil(float64(memoryLimit) / 4.0)) + } + } + fetcher := cFetcherPool.Get().(*cFetcher) fetcher.cFetcherArgs = cFetcherArgs{ spec.LockingStrength, spec.LockingWaitPolicy, flowCtx.EvalCtx.SessionData().LockTimeout, - execinfra.GetWorkMemLimit(flowCtx), + memoryLimit, // Note that the correct estimated row count will be set by the index // joiner for each set of spans to read. 0, /* estimatedRowCount */ false, /* reverse */ flowCtx.TraceKV, } - if err = fetcher.Init(flowCtx.Codec(), fetcherAllocator, kvFetcherMemAcc, tableArgs, spec.HasSystemColumns); err != nil { + if err = fetcher.Init( + flowCtx.Codec(), fetcherAllocator, kvFetcherMemAcc, tableArgs, spec.HasSystemColumns, + ); err != nil { fetcher.Release() return nil, err } @@ -444,8 +508,13 @@ func NewColIndexJoin( spanAssembler: spanAssembler, ResultTypes: tableArgs.typs, maintainOrdering: spec.MaintainOrdering, + usesStreamer: useStreamer, } op.prepareMemLimit(inputTypes) + if useStreamer { + op.streamerInfo.budgetLimit = 3 * memoryLimit + op.streamerInfo.budgetAcc = streamerBudgetAcc + } return op, nil } @@ -528,5 +597,8 @@ func (s *ColIndexJoin) closeInternal() { // spanAssembler can be nil if Release() has already been called. s.spanAssembler.Close() } + if s.streamerInfo.Streamer != nil { + s.streamerInfo.Streamer.Close() + } s.batch = nil } diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index 5f74856199be..8c87630f4eeb 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -87,11 +87,11 @@ func (p *planner) CreateIndex(ctx context.Context, n *tree.CreateIndex) (planNod return &createIndexNode{tableDesc: tableDesc, n: n}, nil } -// setupConstraintForShard adds a check constraint ensuring that the shard +// maybeSetupConstraintForShard adds a check constraint ensuring that the shard // column's value is within [0..ShardBuckets-1]. This method is called when a // `CREATE INDEX`/`ALTER PRIMARY KEY` statement is issued for the creation of a // sharded index that *does not* re-use a pre-existing shard column. -func (p *planner) setupConstraintForShard( +func (p *planner) maybeSetupConstraintForShard( ctx context.Context, tableDesc *tabledesc.Mutable, shardCol catalog.Column, buckets int32, ) error { // Assign an ID to the newly-added shard column, which is needed for the creation @@ -104,31 +104,26 @@ func (p *planner) setupConstraintForShard( if err != nil { return err } - info, err := tableDesc.GetConstraintInfo() + ckBuilder := schemaexpr.MakeCheckConstraintBuilder(ctx, p.tableName, tableDesc, &p.semaCtx) + ckDesc, err := ckBuilder.Build(ckDef) if err != nil { return err } - inuseNames := make(map[string]struct{}, len(info)) - for k := range info { - inuseNames[k] = struct{}{} - } - - ckBuilder := schemaexpr.MakeCheckConstraintBuilder(ctx, p.tableName, tableDesc, &p.semaCtx) - ckName, err := ckBuilder.DefaultName(ckDef.Expr) + curConstraintInfos, err := tableDesc.GetConstraintInfo() if err != nil { return err } // Avoid creating duplicate check constraints. - if _, ok := inuseNames[ckName]; !ok { - ck, err := ckBuilder.Build(ckDef) - if err != nil { - return err + for _, info := range curConstraintInfos { + if info.CheckConstraint != nil && info.CheckConstraint.Expr == ckDesc.Expr { + return nil } - ck.Validity = descpb.ConstraintValidity_Validating - tableDesc.AddCheckMutation(ck, descpb.DescriptorMutation_ADD) } + + ckDesc.Validity = descpb.ConstraintValidity_Validating + tableDesc.AddCheckMutation(ckDesc, descpb.DescriptorMutation_ADD) return nil } @@ -225,7 +220,7 @@ func makeIndexDescriptor( if tableDesc.IsLocalityRegionalByRow() { return nil, hashShardedIndexesOnRegionalByRowError() } - shardCol, newColumns, newColumn, err := setupShardedIndex( + shardCol, newColumns, err := setupShardedIndex( params.ctx, params.EvalContext(), ¶ms.p.semaCtx, @@ -239,10 +234,10 @@ func makeIndexDescriptor( return nil, err } columns = newColumns - if newColumn { - if err := params.p.setupConstraintForShard(params.ctx, tableDesc, shardCol, indexDesc.Sharded.ShardBuckets); err != nil { - return nil, err - } + if err := params.p.maybeSetupConstraintForShard( + params.ctx, tableDesc, shardCol, indexDesc.Sharded.ShardBuckets, + ); err != nil { + return nil, err } } @@ -480,9 +475,10 @@ var hashShardedIndexesDisabledError = pgerror.Newf(pgcode.FeatureNotSupported, "hash sharded indexes require the experimental_enable_hash_sharded_indexes session variable") // setupShardedIndex creates a shard column for the given index descriptor. It -// returns the shard column, the new column list for the index, and a boolean -// which is true if the shard column was newly created. If the shard column is -// new, it is added to tableDesc. +// returns the shard column and the new column list for the index. If the shard +// column is new, either of the following happens: +// (1) the column is added to tableDesc if it's a new table; +// (2) a column mutation is added to tableDesc if the table is not new. func setupShardedIndex( ctx context.Context, evalCtx *tree.EvalContext, @@ -493,9 +489,9 @@ func setupShardedIndex( tableDesc *tabledesc.Mutable, indexDesc *descpb.IndexDescriptor, isNewTable bool, -) (shard catalog.Column, newColumns tree.IndexElemList, newColumn bool, err error) { +) (shard catalog.Column, newColumns tree.IndexElemList, err error) { if !shardedIndexEnabled { - return nil, nil, false, hashShardedIndexesDisabledError + return nil, nil, hashShardedIndexesDisabledError } colNames := make([]string, 0, len(columns)) @@ -504,12 +500,13 @@ func setupShardedIndex( } buckets, err := tabledesc.EvalShardBucketCount(ctx, semaCtx, evalCtx, bucketsExpr) if err != nil { - return nil, nil, false, err + return nil, nil, err } - shardCol, newColumn, err := maybeCreateAndAddShardCol(int(buckets), tableDesc, + shardCol, err := maybeCreateAndAddShardCol(int(buckets), tableDesc, colNames, isNewTable) + if err != nil { - return nil, nil, false, err + return nil, nil, err } shardIdxElem := tree.IndexElem{ Column: tree.Name(shardCol.GetName()), @@ -522,7 +519,7 @@ func setupShardedIndex( ShardBuckets: buckets, ColumnNames: colNames, } - return shardCol, newColumns, newColumn, nil + return shardCol, newColumns, nil } // maybeCreateAndAddShardCol adds a new hidden computed shard column (or its mutation) to @@ -530,10 +527,10 @@ func setupShardedIndex( // buckets. func maybeCreateAndAddShardCol( shardBuckets int, desc *tabledesc.Mutable, colNames []string, isNewTable bool, -) (col catalog.Column, created bool, err error) { +) (col catalog.Column, err error) { shardColDesc, err := makeShardColumnDesc(colNames, shardBuckets) if err != nil { - return nil, false, err + return nil, err } existingShardCol, err := desc.FindColumnWithName(tree.Name(shardColDesc.Name)) if err == nil && !existingShardCol.Dropped() { @@ -543,14 +540,14 @@ func maybeCreateAndAddShardCol( if !existingShardCol.IsHidden() { // The user managed to reverse-engineer our crazy shard column name, so // we'll return an error here rather than try to be tricky. - return nil, false, pgerror.Newf(pgcode.DuplicateColumn, + return nil, pgerror.Newf(pgcode.DuplicateColumn, "column %s already specified; can't be used for sharding", shardColDesc.Name) } - return existingShardCol, false, nil + return existingShardCol, nil } columnIsUndefined := sqlerrors.IsUndefinedColumnError(err) if err != nil && !columnIsUndefined { - return nil, false, err + return nil, err } if columnIsUndefined || existingShardCol.Dropped() { if isNewTable { @@ -558,10 +555,9 @@ func maybeCreateAndAddShardCol( } else { desc.AddColumnMutation(shardColDesc, descpb.DescriptorMutation_ADD) } - created = true } shardCol, err := desc.FindColumnWithName(tree.Name(shardColDesc.Name)) - return shardCol, created, err + return shardCol, err } func (n *createIndexNode) startExec(params runParams) error { diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index c15896f55d8b..5c57239f29a1 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1487,7 +1487,7 @@ func NewTableDesc( if err != nil { return nil, err } - shardCol, _, err := maybeCreateAndAddShardCol(int(buckets), &desc, + shardCol, err := maybeCreateAndAddShardCol(int(buckets), &desc, []string{string(d.Name)}, true, /* isNewTable */ ) if err != nil { @@ -1599,7 +1599,7 @@ func NewTableDesc( if n.PartitionByTable.ContainsPartitions() { return nil, pgerror.New(pgcode.FeatureNotSupported, "sharded indexes don't support partitioning") } - shardCol, newColumns, newColumn, err := setupShardedIndex( + shardCol, newColumns, err := setupShardedIndex( ctx, evalCtx, semaCtx, @@ -1612,18 +1612,39 @@ func NewTableDesc( if err != nil { return nil, err } - if newColumn { - buckets, err := tabledesc.EvalShardBucketCount(ctx, semaCtx, evalCtx, d.Sharded.ShardBuckets) - if err != nil { - return nil, err - } - checkConstraint, err := makeShardCheckConstraintDef(int(buckets), shardCol) - if err != nil { - return nil, err + + buckets, err := tabledesc.EvalShardBucketCount(ctx, semaCtx, evalCtx, d.Sharded.ShardBuckets) + if err != nil { + return nil, err + } + checkConstraint, err := makeShardCheckConstraintDef(int(buckets), shardCol) + if err != nil { + return nil, err + } + + // If there is an equivalent check constraint from the CREATE TABLE (should + // be rare since we hide the constraint of shard column), we don't create a + // duplicate one. + ckBuilder := schemaexpr.MakeCheckConstraintBuilder(ctx, n.Table, &desc, semaCtx) + checkConstraintDesc, err := ckBuilder.Build(checkConstraint) + if err != nil { + return nil, err + } + for _, def := range n.Defs { + if inputCheckConstraint, ok := def.(*tree.CheckConstraintTableDef); ok { + inputCheckConstraintDesc, err := ckBuilder.Build(inputCheckConstraint) + if err != nil { + return nil, err + } + if checkConstraintDesc.Expr == inputCheckConstraintDesc.Expr { + return newColumns, nil + } } - n.Defs = append(n.Defs, checkConstraint) - cdd = append(cdd, nil) } + + n.Defs = append(n.Defs, checkConstraint) + cdd = append(cdd, nil) + return newColumns, nil } diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index ab2caf46aaf5..144b22d61725 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -408,8 +408,19 @@ func (ds *ServerImpl) setupFlow( // that have no remote flows and also no concurrency, the txn comes from // localState.Txn. Otherwise, we create a txn based on the request's // LeafTxnInputState. + useLeaf := false + for _, proc := range req.Flow.Processors { + if jr := proc.Core.JoinReader; jr != nil { + if !jr.MaintainOrdering && jr.IsIndexJoin() { + // Index joins when ordering doesn't have to be maintained are + // executed via the Streamer API that has concurrency. + useLeaf = true + break + } + } + } var txn *kv.Txn - if localState.IsLocal && !f.ConcurrentTxnUse() { + if localState.IsLocal && !f.ConcurrentTxnUse() && !useLeaf { txn = localState.Txn } else { // If I haven't created the leaf already, do it now. diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 2e7b16dcd94e..e61ca87a4762 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -447,6 +447,16 @@ func (dsp *DistSQLPlanner) Run( localState.HasConcurrency = localState.HasConcurrency || execinfra.HasParallelProcessors(flow) } } + for _, proc := range plan.Processors { + if js := proc.Spec.Core.JoinReader; js != nil { + if !js.MaintainOrdering && js.IsIndexJoin() { + // Index joins when ordering doesn't have to be maintained + // are executed via the Streamer API that has concurrency. + localState.HasConcurrency = true + break + } + } + } } if localState.MustUseLeafTxn() && txn != nil { // Set up leaf txns using the txnCoordMeta if we need to. diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 4f3629b3a9b0..731ea24937ca 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/jobs", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvclient/rangecache:with-mocks", "//pkg/kv/kvserver/diskmap", "//pkg/kv/kvserver/kvserverbase", diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 9eeda739e7d6..1e405372998e 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/diskmap" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" @@ -145,6 +146,8 @@ type ServerConfig struct { // AdminVerifyProtectedTimestampRequest. ProtectedTimestampProvider protectedts.Provider + DistSender *kvcoord.DistSender + // RangeCache is used by processors that were supposed to have been planned on // the leaseholders of the data ranges that they're consuming. These // processors query the cache to see if they should communicate updates to the diff --git a/pkg/sql/execinfrapb/processors.go b/pkg/sql/execinfrapb/processors.go index 045e7f583e1c..62c2a499cba4 100644 --- a/pkg/sql/execinfrapb/processors.go +++ b/pkg/sql/execinfrapb/processors.go @@ -499,3 +499,9 @@ func (spec *WindowerSpec_Frame) ConvertToAST() (*tree.WindowFrame, error) { Exclusion: exclusion, }, nil } + +// IsIndexJoin returns true if spec defines an index join (as opposed to a +// lookup join). +func (spec *JoinReaderSpec) IsIndexJoin() bool { + return len(spec.LookupColumns) == 0 && spec.LookupExpr.Empty() +} diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index d8a3d45049f8..60356dc30007 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -218,8 +218,7 @@ t CREATE TABLE public.t ( UNIQUE INDEX i5 (w ASC) STORING (y), INVERTED INDEX i6 (v), INDEX i7 (z ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY fam_0_x_y_z_w_v (x, y, z, w, v), - CONSTRAINT check_crdb_internal_z_shard_4 CHECK (crdb_internal_z_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY fam_0_x_y_z_w_v (x, y, z, w, v) ) # Test that the indexes we expect got rewritten. All but i3 should have been rewritten, @@ -368,9 +367,7 @@ t CREATE TABLE public.t ( CONSTRAINT t_pkey PRIMARY KEY (y ASC) USING HASH WITH BUCKET_COUNT = 10, UNIQUE INDEX t_x_key (x ASC), INDEX i1 (z ASC) USING HASH WITH BUCKET_COUNT = 5, - FAMILY fam_0_x_y_z (x, y, z), - CONSTRAINT check_crdb_internal_z_shard_5 CHECK (crdb_internal_z_shard_5 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8)), - CONSTRAINT check_crdb_internal_y_shard_10 CHECK (crdb_internal_y_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY fam_0_x_y_z (x, y, z) ) query T @@ -429,8 +426,7 @@ t CREATE TABLE public.t ( CONSTRAINT t_pkey PRIMARY KEY (y ASC), UNIQUE INDEX t_x_key (x ASC) USING HASH WITH BUCKET_COUNT = 5, INDEX i (z ASC), - FAMILY fam_0_x_y_z (x, y, z), - CONSTRAINT check_crdb_internal_x_shard_5 CHECK (crdb_internal_x_shard_5 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8)) + FAMILY fam_0_x_y_z (x, y, z) ) query III @@ -556,8 +552,7 @@ t CREATE TABLE public.t ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), crdb_internal_x_shard_4 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(x)), 4:::INT8)) VIRTUAL, CONSTRAINT t_pkey PRIMARY KEY (x ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (x, rowid), - CONSTRAINT check_crdb_internal_x_shard_4 CHECK (crdb_internal_x_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (x, rowid) ) statement ok @@ -950,9 +945,7 @@ t CREATE TABLE public.t ( x INT8 NOT NULL, crdb_internal_x_shard_3 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(x)), 3:::INT8)) VIRTUAL, CONSTRAINT t_pkey PRIMARY KEY (x ASC) USING HASH WITH BUCKET_COUNT = 3, - FAMILY "primary" (x), - CONSTRAINT check_crdb_internal_x_shard_2 CHECK (crdb_internal_x_shard_2 IN (0:::INT8, 1:::INT8)), - CONSTRAINT check_crdb_internal_x_shard_3 CHECK (crdb_internal_x_shard_3 IN (0:::INT8, 1:::INT8, 2:::INT8)) + FAMILY "primary" (x) ) # Changes on a hash sharded index that change the columns will cause the old @@ -972,9 +965,7 @@ t CREATE TABLE public.t ( crdb_internal_y_shard_2 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(y)), 2:::INT8)) VIRTUAL, CONSTRAINT t_pkey PRIMARY KEY (y ASC) USING HASH WITH BUCKET_COUNT = 2, UNIQUE INDEX t_x_key (x ASC) USING HASH WITH BUCKET_COUNT = 2, - FAMILY fam_0_x_y (x, y), - CONSTRAINT check_crdb_internal_x_shard_2 CHECK (crdb_internal_x_shard_2 IN (0:::INT8, 1:::INT8)), - CONSTRAINT check_crdb_internal_y_shard_2 CHECK (crdb_internal_y_shard_2 IN (0:::INT8, 1:::INT8)) + FAMILY fam_0_x_y (x, y) ) # Regression for #49079. diff --git a/pkg/sql/logictest/testdata/logic_test/create_table b/pkg/sql/logictest/testdata/logic_test/create_table index be9cf56d1639..6d9201d2964f 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_table +++ b/pkg/sql/logictest/testdata/logic_test/create_table @@ -369,8 +369,7 @@ like_hash CREATE TABLE public.like_hash ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT like_hash_base_pkey PRIMARY KEY (rowid ASC), INDEX like_hash_base_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index index 02ad7092853b..9c6343a80c39 100644 --- a/pkg/sql/logictest/testdata/logic_test/hash_sharded_index +++ b/pkg/sql/logictest/testdata/logic_test/hash_sharded_index @@ -12,8 +12,7 @@ sharded_primary CREATE TABLE public.sharded_primary ( crdb_internal_a_shard_10 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 10:::INT8)) VIRTUAL, a INT8 NOT NULL, CONSTRAINT sharded_primary_pkey PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a) ) statement error pgcode 22023 BUCKET_COUNT must be a 32-bit integer greater than 1, got -1 @@ -49,8 +48,7 @@ sharded_primary CREATE TABLE public.sharded_primary ( a INT8 NOT NULL, crdb_internal_a_shard_10 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 10:::INT8)) VIRTUAL, CONSTRAINT "primary" PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a) ) query TTT colnames @@ -107,8 +105,7 @@ specific_family CREATE TABLE public.specific_family ( CONSTRAINT specific_family_pkey PRIMARY KEY (rowid ASC), INDEX specific_family_b_idx (b ASC) USING HASH WITH BUCKET_COUNT = 10, FAMILY a_family (a, rowid), - FAMILY b_family (b), - CONSTRAINT check_crdb_internal_b_shard_10 CHECK (crdb_internal_b_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY b_family (b) ) # Tests for secondary sharded indexes @@ -124,8 +121,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT sharded_secondary_pkey PRIMARY KEY (rowid ASC), INDEX sharded_secondary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -147,8 +143,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT sharded_secondary_pkey PRIMARY KEY (rowid ASC), INDEX sharded_secondary_crdb_internal_a_shard_4_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -177,8 +172,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( crdb_internal_a_shard_10 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 10:::INT8)) VIRTUAL, CONSTRAINT sharded_secondary_pkey PRIMARY KEY (rowid ASC), INDEX sharded_secondary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -199,9 +193,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( CONSTRAINT sharded_secondary_pkey PRIMARY KEY (rowid ASC), INDEX sharded_secondary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 10, INDEX sharded_secondary_a_idx1 (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a, rowid) ) # Drop a sharded index and ensure that the shard column is dropped with it. @@ -217,8 +209,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( crdb_internal_a_shard_4 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 4:::INT8)) VIRTUAL, CONSTRAINT sharded_secondary_pkey PRIMARY KEY (rowid ASC), INDEX sharded_secondary_a_idx1 (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -279,8 +270,7 @@ sharded_secondary CREATE TABLE public.sharded_secondary ( INDEX sharded_secondary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 10, INDEX sharded_secondary_a_idx1 (a ASC) USING HASH WITH BUCKET_COUNT = 10, INDEX sharded_secondary_a_idx2 (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a, rowid) ) @@ -302,9 +292,7 @@ sharded_primary CREATE TABLE public.sharded_primary ( crdb_internal_a_shard_4 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 4:::INT8)) VIRTUAL, CONSTRAINT "primary" PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 10, INDEX sharded_primary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" (a), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)), - CONSTRAINT check_crdb_internal_a_shard_4 CHECK (crdb_internal_a_shard_4 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" (a) ) statement ok @@ -320,8 +308,7 @@ sharded_primary CREATE TABLE public.sharded_primary ( a INT8 NOT NULL, crdb_internal_a_shard_10 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 10:::INT8)) VIRTUAL, CONSTRAINT "primary" PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a) ) statement ok @@ -335,8 +322,7 @@ sharded_primary CREATE TABLE public.sharded_primary ( crdb_internal_a_shard_10 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 10:::INT8)) VIRTUAL, CONSTRAINT "primary" PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 10, INDEX sharded_primary_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 10, - FAMILY "primary" (a), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a) ) statement ok @@ -416,8 +402,7 @@ column_used_on_unsharded CREATE TABLE public.column_used_on_unsharded ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT column_used_on_unsharded_pkey PRIMARY KEY (rowid ASC), INDEX column_used_on_unsharded_crdb_internal_a_shard_10_idx (crdb_internal_a_shard_10 ASC), - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -442,8 +427,7 @@ column_used_on_unsharded_create_table CREATE TABLE public.column_used_on_unshar rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT column_used_on_unsharded_create_table_pkey PRIMARY KEY (rowid ASC), INDEX column_used_on_unsharded_create_table_crdb_internal_a_shard_10_idx (crdb_internal_a_shard_10 ASC), - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_10 CHECK (crdb_internal_a_shard_10 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8)) + FAMILY "primary" (a, rowid) ) statement ok @@ -499,9 +483,7 @@ weird_names CREATE TABLE public.weird_names ( "crdb_internal_'quotes' in the column's name_shard_4" INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes("'quotes' in the column's name")), 4:::INT8)) VIRTUAL, CONSTRAINT weird_names_pkey PRIMARY KEY ("I am a column with spaces" ASC) USING HASH WITH BUCKET_COUNT = 12, INDEX foo ("'quotes' in the column's name" ASC) USING HASH WITH BUCKET_COUNT = 4, - FAMILY "primary" ("I am a column with spaces", "'quotes' in the column's name"), - CONSTRAINT "check_crdb_internal_I am a column with spaces_shard_12" CHECK ("crdb_internal_I am a column with spaces_shard_12" IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8)), - CONSTRAINT "check_crdb_internal_'quotes' in the column's name_shard_4" CHECK ("crdb_internal_'quotes' in the column's name_shard_4" IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8)) + FAMILY "primary" ("I am a column with spaces", "'quotes' in the column's name") ) subtest column_does_not_exist @@ -577,9 +559,7 @@ rename_column CREATE TABLE public.rename_column ( crdb_internal_c2_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(c2)), 8:::INT8)) VIRTUAL, CONSTRAINT rename_column_pkey PRIMARY KEY (c0 ASC, c1 ASC) USING HASH WITH BUCKET_COUNT = 8, INDEX rename_column_c2_idx (c2 ASC) USING HASH WITH BUCKET_COUNT = 8, - FAMILY "primary" (c0, c1, c2), - CONSTRAINT check_crdb_internal_c0_c1_shard_8 CHECK (crdb_internal_c0_c1_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)), - CONSTRAINT check_crdb_internal_c2_shard_8 CHECK (crdb_internal_c2_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + FAMILY "primary" (c0, c1, c2) ) statement ok @@ -603,9 +583,7 @@ rename_column CREATE TABLE public.rename_column ( crdb_internal_c3_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(c3)), 8:::INT8)) VIRTUAL, CONSTRAINT rename_column_pkey PRIMARY KEY (c1 ASC, c2 ASC) USING HASH WITH BUCKET_COUNT = 8, INDEX rename_column_c2_idx (c3 ASC) USING HASH WITH BUCKET_COUNT = 8, - FAMILY "primary" (c1, c2, c3), - CONSTRAINT check_crdb_internal_c0_c1_shard_8 CHECK (crdb_internal_c1_c2_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)), - CONSTRAINT check_crdb_internal_c2_shard_8 CHECK (crdb_internal_c3_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + FAMILY "primary" (c1, c2, c3) ) query III @@ -628,9 +606,7 @@ rename_column CREATE TABLE public.rename_column ( crdb_internal_c2_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(c2)), 8:::INT8)) VIRTUAL, CONSTRAINT rename_column_pkey PRIMARY KEY (c0 ASC, c1 ASC) USING HASH WITH BUCKET_COUNT = 8, INDEX rename_column_c2_idx (c2 ASC) USING HASH WITH BUCKET_COUNT = 8, - FAMILY "primary" (c0, c1, c2), - CONSTRAINT check_crdb_internal_c0_c1_shard_8 CHECK (crdb_internal_c0_c1_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)), - CONSTRAINT check_crdb_internal_c2_shard_8 CHECK (crdb_internal_c2_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + FAMILY "primary" (c0, c1, c2) ) query III @@ -807,3 +783,117 @@ INSERT INTO parent VALUES (1,1) statement ok INSERT INTO child VALUES (1,1) + +# Test creating tables with output of `SHOW CREATE TABLE` from table with +# hash-sharded index and make sure constraint of shard column is preserved and +# recognized by optimizer plan +subtest create_with_show_create + +statement ok +DROP TABLE IF EXISTS t + +statement ok +CREATE TABLE t ( + a INT PRIMARY KEY USING HASH WITH BUCKET_COUNT = 8 +); + +query T +explain (opt, catalog) select * from t +---- +TABLE t + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32("crdb_internal.datums_to_bytes"(a)), 8:::INT8)) virtual [hidden] + ├── a int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] + ├── tableoid oid [hidden] [system] + ├── CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + └── PRIMARY INDEX t_pkey + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32("crdb_internal.datums_to_bytes"(a)), 8:::INT8)) virtual [hidden] + └── a int not null + scan t + ├── check constraint expressions + │ └── crdb_internal_a_shard_8 IN (0, 1, 2, 3, 4, 5, 6, 7) + └── computed column expressions + └── crdb_internal_a_shard_8 + └── mod(fnv32(crdb_internal.datums_to_bytes(a)), 8) + +let $create_statement +SELECT create_statement FROM [SHOW CREATE TABLE t] + +statement ok +DROP TABLE t + +statement ok +$create_statement + +query T +SELECT @2 FROM [SHOW CREATE TABLE t] +---- +CREATE TABLE public.t ( + crdb_internal_a_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) VIRTUAL, + a INT8 NOT NULL, + CONSTRAINT t_pkey PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 8, + FAMILY "primary" (a) +) + +query T +explain (opt, catalog) select * from t +---- +TABLE t + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) virtual [hidden] + ├── a int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] + ├── tableoid oid [hidden] [system] + ├── CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + └── PRIMARY INDEX t_pkey + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) virtual [hidden] + └── a int not null + scan t + ├── check constraint expressions + │ └── crdb_internal_a_shard_8 IN (0, 1, 2, 3, 4, 5, 6, 7) + └── computed column expressions + └── crdb_internal_a_shard_8 + └── mod(fnv32(crdb_internal.datums_to_bytes(a)), 8) + +# Make sure user defined constraint is used if it's equivalent to the shard +# column constraint would have been created. +statement ok +DROP TABLE t + +statement ok +CREATE TABLE public.t ( + crdb_internal_a_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) VIRTUAL, + a INT8 NOT NULL, + CONSTRAINT t_pkey PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 8, + FAMILY "primary" (a), + CONSTRAINT check_crdb_internal_a_shard_8 CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) +) + +query T +SELECT @2 FROM [SHOW CREATE TABLE t] +---- +CREATE TABLE public.t ( + crdb_internal_a_shard_8 INT4 NOT VISIBLE NOT NULL AS (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) VIRTUAL, + a INT8 NOT NULL, + CONSTRAINT t_pkey PRIMARY KEY (a ASC) USING HASH WITH BUCKET_COUNT = 8, + FAMILY "primary" (a), + CONSTRAINT check_crdb_internal_a_shard_8 CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) +) + +query T +explain (opt, catalog) select * from t +---- +TABLE t + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) virtual [hidden] + ├── a int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] + ├── tableoid oid [hidden] [system] + ├── CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + └── PRIMARY INDEX t_pkey + ├── crdb_internal_a_shard_8 int4 not null as (mod(fnv32(crdb_internal.datums_to_bytes(a)), 8:::INT8)) virtual [hidden] + └── a int not null + scan t + ├── check constraint expressions + │ └── crdb_internal_a_shard_8 IN (0, 1, 2, 3, 4, 5, 6, 7) + └── computed column expressions + └── crdb_internal_a_shard_8 + └── mod(fnv32(crdb_internal.datums_to_bytes(a)), 8) diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index 77250db03c06..77fe5d008a13 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "helper.go", "inserter.go", "kv_batch_fetcher.go", + "kv_batch_streamer.go", "kv_fetcher.go", "locking.go", "metrics.go", @@ -23,14 +24,17 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/row", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvstreamer", "//pkg/kv/kvserver", "//pkg/kv/kvserver/concurrency/lock", "//pkg/roachpb:with-mocks", "//pkg/settings", + "//pkg/settings/cluster", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", diff --git a/pkg/sql/row/kv_batch_streamer.go b/pkg/sql/row/kv_batch_streamer.go new file mode 100644 index 000000000000..c9503dea9325 --- /dev/null +++ b/pkg/sql/row/kv_batch_streamer.go @@ -0,0 +1,213 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package row + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// CanUseStreamer returns whether the kvstreamer.Streamer API should be used. +func CanUseStreamer(ctx context.Context, settings *cluster.Settings) bool { + // TODO(yuzefovich): remove the version gate in 22.2 cycle. + return settings.Version.IsActive(ctx, clusterversion.TargetBytesAvoidExcess) && + useStreamerEnabled.Get(&settings.SV) +} + +// useStreamerEnabled determines whether the Streamer API should be used. +// TODO(yuzefovich): remove this in 22.2. +var useStreamerEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.distsql.use_streamer.enabled", + "determines whether the usage of the Streamer API is allowed. "+ + "Enabling this will increase the speed of lookup/index joins "+ + "while adhering to memory limits.", + true, +) + +// TxnKVStreamer handles retrieval of key/values. +type TxnKVStreamer struct { + streamer *kvstreamer.Streamer + spans roachpb.Spans + + // numOutstandingRequests tracks the number of requests that haven't been + // fully responded to yet. + numOutstandingRequests int + + results []kvstreamer.Result + lastResultState struct { + kvstreamer.Result + // numEmitted tracks the number of times this result has been fully + // emitted. + numEmitted int + // Used only for ScanResponses. + remainingBatches [][]byte + } +} + +var _ KVBatchFetcher = &TxnKVStreamer{} + +// NewTxnKVStreamer creates a new TxnKVStreamer. +func NewTxnKVStreamer( + ctx context.Context, + streamer *kvstreamer.Streamer, + spans roachpb.Spans, + lockStrength descpb.ScanLockingStrength, +) (*TxnKVStreamer, error) { + if log.ExpensiveLogEnabled(ctx, 2) { + log.VEventf(ctx, 2, "Scan %s", spans) + } + keyLocking := getKeyLockingStrength(lockStrength) + reqs := spansToRequests(spans, false /* reverse */, keyLocking) + if err := streamer.Enqueue(ctx, reqs, nil /* enqueueKeys */); err != nil { + return nil, err + } + return &TxnKVStreamer{ + streamer: streamer, + spans: spans, + numOutstandingRequests: len(spans), + }, nil +} + +// proceedWithLastResult processes the result which must be already set on the +// lastResultState and emits the first part of the response (the only part for +// GetResponses). +func (f *TxnKVStreamer) proceedWithLastResult( + ctx context.Context, +) (skip bool, kvs []roachpb.KeyValue, batchResp []byte, err error) { + result := f.lastResultState.Result + if get := result.GetResp; get != nil { + if get.IntentValue != nil { + return false, nil, nil, errors.AssertionFailedf( + "unexpectedly got an IntentValue back from a SQL GetRequest %v", *get.IntentValue, + ) + } + if get.Value == nil { + // Nothing found in this particular response, so we skip it. + f.releaseLastResult(ctx) + return true, nil, nil, nil + } + pos := result.EnqueueKeysSatisfied[f.lastResultState.numEmitted] + origSpan := f.spans[pos] + f.lastResultState.numEmitted++ + f.numOutstandingRequests-- + return false, []roachpb.KeyValue{{Key: origSpan.Key, Value: *get.Value}}, nil, nil + } + scan := result.ScanResp + if len(scan.BatchResponses) > 0 { + batchResp, f.lastResultState.remainingBatches = scan.BatchResponses[0], scan.BatchResponses[1:] + } + if len(f.lastResultState.remainingBatches) == 0 { + f.processedScanResponse() + } + return false, scan.Rows, batchResp, nil +} + +// processedScanResponse updates the lastResultState before emitting the last +// part of the ScanResponse. This method should be called for each request that +// the ScanResponse satisfies. +func (f *TxnKVStreamer) processedScanResponse() { + f.lastResultState.numEmitted++ + if f.lastResultState.ScanResp.Complete { + f.numOutstandingRequests-- + } +} + +func (f *TxnKVStreamer) releaseLastResult(ctx context.Context) { + f.lastResultState.MemoryTok.Release(ctx) + f.lastResultState.Result = kvstreamer.Result{} +} + +// nextBatch returns the next batch of key/value pairs. If there are none +// available, a fetch is initiated. When there are no more keys, ok is false. +func (f *TxnKVStreamer) nextBatch( + ctx context.Context, +) (ok bool, kvs []roachpb.KeyValue, batchResp []byte, err error) { + if f.numOutstandingRequests == 0 { + // All requests have already been responded to. + f.releaseLastResult(ctx) + return false, nil, nil, nil + } + + // Check whether there are more batches in the current ScanResponse. + if len(f.lastResultState.remainingBatches) > 0 { + batchResp, f.lastResultState.remainingBatches = f.lastResultState.remainingBatches[0], f.lastResultState.remainingBatches[1:] + if len(f.lastResultState.remainingBatches) == 0 { + f.processedScanResponse() + } + return true, nil, batchResp, nil + } + + // Check whether the current result satisfies multiple requests. + if f.lastResultState.numEmitted < len(f.lastResultState.EnqueueKeysSatisfied) { + // Note that we should never get an error here since we're processing + // the same result again. + _, kvs, batchResp, err = f.proceedWithLastResult(ctx) + return true, kvs, batchResp, err + } + + // Release the current result. + if f.lastResultState.numEmitted == len(f.lastResultState.EnqueueKeysSatisfied) && f.lastResultState.numEmitted > 0 { + f.releaseLastResult(ctx) + } + + // Process the next result we have already received from the streamer. + for len(f.results) > 0 { + // Peel off the next result and set it into lastResultState. + f.lastResultState.Result = f.results[0] + f.lastResultState.numEmitted = 0 + f.lastResultState.remainingBatches = nil + // Lose the reference to that result and advance the results slice for + // the next iteration. + f.results[0] = kvstreamer.Result{} + f.results = f.results[1:] + var skip bool + skip, kvs, batchResp, err = f.proceedWithLastResult(ctx) + if err != nil { + return false, nil, nil, err + } + if skip { + continue + } + return true, kvs, batchResp, nil + } + + // Get more results from the streamer. This call will block until some + // results are available or we're done. + // + // The memory accounting for the returned results has already been performed + // by the streamer against its own budget, so we don't have to concern + // ourselves with the memory accounting here. + f.results, err = f.streamer.GetResults(ctx) + if len(f.results) == 0 || err != nil { + return false, nil, nil, err + } + return f.nextBatch(ctx) +} + +// close releases the resources of this TxnKVStreamer. +func (f *TxnKVStreamer) close(ctx context.Context) { + if f.lastResultState.MemoryTok != nil { + f.lastResultState.MemoryTok.Release(ctx) + } + for _, r := range f.results { + r.MemoryTok.Release(ctx) + } + *f = TxnKVStreamer{} +} diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 25eac1ef211a..7c0ed1b164f5 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -108,11 +108,18 @@ func NewKVFetcher( return newKVFetcher(&kvBatchFetcher), err } +// NewKVStreamingFetcher returns a new KVFetcher that utilizes the provided +// TxnKVStreamer to perform KV reads. +func NewKVStreamingFetcher(streamer *TxnKVStreamer) *KVFetcher { + return &KVFetcher{ + KVBatchFetcher: streamer, + } +} + func newKVFetcher(batchFetcher KVBatchFetcher) *KVFetcher { - ret := &KVFetcher{ + return &KVFetcher{ KVBatchFetcher: batchFetcher, } - return ret } // GetBytesRead returns the number of bytes read by this fetcher. It is safe for diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index af23fb59fe02..abd6579165cd 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -45,6 +45,8 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvclient/kvstreamer", + "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb:with-mocks", "//pkg/server/telemetry", @@ -184,6 +186,7 @@ go_test( "//pkg/util/mon", "//pkg/util/protoutil", "//pkg/util/randutil", + "//pkg/util/stop", "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 6fd2a8430365..9400612d81a8 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -12,9 +12,12 @@ package rowexec import ( "context" + "math" "sort" "unsafe" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -115,6 +118,19 @@ type joinReader struct { shouldLimitBatches bool readerType joinReaderType + keyLocking descpb.ScanLockingStrength + lockWaitPolicy lock.WaitPolicy + + // usesStreamer indicates whether the joinReader performs the lookups using + // the kvcoord.Streamer API. + usesStreamer bool + streamerInfo struct { + *kvstreamer.Streamer + unlimitedMemMonitor *mon.BytesMonitor + budgetAcc mon.BoundAccount + budgetLimit int64 + } + input execinfra.RowSource // lookupCols and lookupExpr (and optionally remoteLookupExpr) represent the @@ -145,6 +161,11 @@ type joinReader struct { batchSizeBytes int64 curBatchSizeBytes int64 + // pendingRow tracks the row that has already been read from the input but + // was not included into the lookup batch because it would make the batch + // exceed batchSizeBytes. + pendingRow rowenc.EncDatumRow + // rowsRead is the total number of rows that this fetcher read from // disk. rowsRead int64 @@ -280,6 +301,8 @@ func newJoinReader( if flowCtx.EvalCtx.SessionData().ParallelizeMultiKeyLookupJoinsEnabled { shouldLimitBatches = false } + tryStreamer := row.CanUseStreamer(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Settings) && !spec.MaintainOrdering + jr := &joinReader{ desc: tableDesc, maintainOrdering: spec.MaintainOrdering, @@ -288,6 +311,9 @@ func newJoinReader( outputGroupContinuationForLeftRow: spec.OutputGroupContinuationForLeftRow, shouldLimitBatches: shouldLimitBatches, readerType: readerType, + keyLocking: spec.LockingStrength, + lockWaitPolicy: row.GetWaitPolicy(spec.LockingWaitPolicy), + usesStreamer: (readerType == indexJoinReaderType) && tryStreamer, lookupBatchBytesLimit: rowinfra.BytesLimit(spec.LookupBatchBytesLimit), } if readerType != indexJoinReaderType { @@ -438,6 +464,32 @@ func newJoinReader( } jr.batchSizeBytes = jr.strategy.getLookupRowsBatchSizeHint(flowCtx.EvalCtx.SessionData()) + if jr.usesStreamer { + maxKeysPerRow, err := jr.desc.KeysPerRow(jr.index.GetID()) + if err != nil { + return nil, err + } + if maxKeysPerRow > 1 { + // Currently, the streamer only supports cases with a single column + // family. + jr.usesStreamer = false + } else { + // jr.batchSizeBytes will be used up by the input batch, and we'll + // give everything else to the streamer budget. Note that + // budgetLimit will always be positive given that memoryLimit is at + // least 8MiB and batchSizeBytes is at most 4MiB. + jr.streamerInfo.budgetLimit = memoryLimit - jr.batchSizeBytes + // We need to use an unlimited monitor for the streamer's budget + // since the streamer itself is responsible for staying under the + // limit. + jr.streamerInfo.unlimitedMemMonitor = mon.NewMonitorInheritWithLimit( + "joinreader-streamer-unlimited" /* name */, math.MaxInt64, flowCtx.EvalCtx.Mon, + ) + jr.streamerInfo.unlimitedMemMonitor.Start(flowCtx.EvalCtx.Ctx(), flowCtx.EvalCtx.Mon, mon.BoundAccount{}) + jr.streamerInfo.budgetAcc = jr.streamerInfo.unlimitedMemMonitor.MakeBoundAccount() + } + } + // TODO(radu): verify the input types match the index key types return jr, nil } @@ -704,28 +756,53 @@ func (jr *joinReader) readInput() ( } // Read the next batch of input rows. - for jr.curBatchSizeBytes < jr.batchSizeBytes { - row, meta := jr.input.Next() - if meta != nil { - if meta.Err != nil { - jr.MoveToDraining(nil /* err */) - return jrStateUnknown, nil, meta + for { + var encDatumRow rowenc.EncDatumRow + var rowSize int64 + if jr.pendingRow == nil { + // There is no pending row, so we have to get the next one from the + // input. + var meta *execinfrapb.ProducerMetadata + encDatumRow, meta = jr.input.Next() + if meta != nil { + if meta.Err != nil { + jr.MoveToDraining(nil /* err */) + return jrStateUnknown, nil, meta + } + + if err := jr.performMemoryAccounting(); err != nil { + jr.MoveToDraining(err) + return jrStateUnknown, nil, meta + } + + return jrReadingInput, nil, meta } - - if err := jr.performMemoryAccounting(); err != nil { - jr.MoveToDraining(err) - return jrStateUnknown, nil, meta + if encDatumRow == nil { + break } - - return jrReadingInput, nil, meta - } - if row == nil { - break + rowSize = int64(encDatumRow.Size()) + if jr.curBatchSizeBytes > 0 && jr.curBatchSizeBytes+rowSize > jr.batchSizeBytes { + // Adding this row to the current batch will make the batch + // exceed jr.batchSizeBytes. Additionally, the batch is not + // empty, so we'll store this row as "pending" and will include + // it into the next batch. + // + // The batch being non-empty is important because in case it was + // empty and we decided to not include this (first) row into it, + // then we'd be stalled - we'd generate no spans, so we'd not + // perform the lookup of anything. + jr.pendingRow = encDatumRow + break + } + } else { + encDatumRow = jr.pendingRow + jr.pendingRow = nil + rowSize = int64(encDatumRow.Size()) } - jr.curBatchSizeBytes += int64(row.Size()) + jr.curBatchSizeBytes += rowSize if jr.groupingState != nil { // Lookup Join. - if err := jr.processContinuationValForRow(row); err != nil { + if err := jr.processContinuationValForRow(encDatumRow); err != nil { jr.MoveToDraining(err) return jrStateUnknown, nil, jr.DrainHelper() } @@ -734,12 +811,11 @@ func (jr *joinReader) readInput() ( // // We need to subtract the EncDatumRowOverhead because that is already // tracked in jr.accountedFor.scratchInputRows. - rowSize := int64(row.Size() - rowenc.EncDatumRowOverhead) - if err := jr.memAcc.Grow(jr.Ctx, rowSize); err != nil { + if err := jr.memAcc.Grow(jr.Ctx, rowSize-int64(rowenc.EncDatumRowOverhead)); err != nil { jr.MoveToDraining(err) return jrStateUnknown, nil, jr.DrainHelper() } - jr.scratchInputRows = append(jr.scratchInputRows, jr.rowAlloc.CopyRow(row)) + jr.scratchInputRows = append(jr.scratchInputRows, jr.rowAlloc.CopyRow(encDatumRow)) } if err := jr.performMemoryAccounting(); err != nil { @@ -810,24 +886,35 @@ func (jr *joinReader) readInput() ( } log.VEventf(jr.Ctx, 1, "scanning %d spans", len(spans)) - var bytesLimit rowinfra.BytesLimit - if !jr.shouldLimitBatches { - bytesLimit = rowinfra.NoBytesLimit - } else { - bytesLimit = jr.lookupBatchBytesLimit - if jr.lookupBatchBytesLimit == 0 { - bytesLimit = rowinfra.DefaultBatchBytesLimit - } - } // Note that the fetcher takes ownership of the spans slice - it will modify // it and perform the memory accounting. We don't care about the // modification here, but we want to be conscious about the memory // accounting - we don't double count for any memory of spans because the // joinReaderStrategy doesn't account for any memory used by the spans. - if err := jr.fetcher.StartScan( - jr.Ctx, jr.FlowCtx.Txn, spans, bytesLimit, rowinfra.NoRowLimit, - jr.FlowCtx.TraceKV, jr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, - ); err != nil { + if jr.usesStreamer { + var kvBatchFetcher *row.TxnKVStreamer + kvBatchFetcher, err = row.NewTxnKVStreamer(jr.Ctx, jr.streamerInfo.Streamer, spans, jr.keyLocking) + if err != nil { + jr.MoveToDraining(err) + return jrStateUnknown, nil, jr.DrainHelper() + } + err = jr.fetcher.StartScanFrom(jr.Ctx, kvBatchFetcher, jr.FlowCtx.TraceKV) + } else { + var bytesLimit rowinfra.BytesLimit + if !jr.shouldLimitBatches { + bytesLimit = rowinfra.NoBytesLimit + } else { + bytesLimit = jr.lookupBatchBytesLimit + if jr.lookupBatchBytesLimit == 0 { + bytesLimit = rowinfra.DefaultBatchBytesLimit + } + } + err = jr.fetcher.StartScan( + jr.Ctx, jr.FlowCtx.Txn, spans, bytesLimit, rowinfra.NoRowLimit, + jr.FlowCtx.TraceKV, jr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, + ) + } + if err != nil { jr.MoveToDraining(err) return jrStateUnknown, nil, jr.DrainHelper() } @@ -942,6 +1029,21 @@ func (jr *joinReader) performMemoryAccounting() error { func (jr *joinReader) Start(ctx context.Context) { ctx = jr.StartInternal(ctx, joinReaderProcName) jr.input.Start(ctx) + if jr.usesStreamer { + jr.streamerInfo.Streamer = kvstreamer.NewStreamer( + jr.FlowCtx.Cfg.DistSender, + jr.FlowCtx.Stopper(), + jr.FlowCtx.Txn, + jr.FlowCtx.EvalCtx.Settings, + jr.lockWaitPolicy, + jr.streamerInfo.budgetLimit, + &jr.streamerInfo.budgetAcc, + ) + jr.streamerInfo.Streamer.Init( + kvstreamer.OutOfOrder, + kvstreamer.Hints{UniqueRequests: true}, + ) + } jr.runningState = jrReadingInput } @@ -956,6 +1058,16 @@ func (jr *joinReader) close() { if jr.fetcher != nil { jr.fetcher.Close(jr.Ctx) } + if jr.usesStreamer { + // We have to cleanup the streamer after closing the fetcher because + // the latter might release some memory tracked by the budget of the + // streamer. + if jr.streamerInfo.Streamer != nil { + jr.streamerInfo.Streamer.Close() + } + jr.streamerInfo.budgetAcc.Close(jr.Ctx) + jr.streamerInfo.unlimitedMemMonitor.Stop(jr.Ctx) + } jr.strategy.close(jr.Ctx) jr.memAcc.Close(jr.Ctx) if jr.limitedMemMonitor != nil { diff --git a/pkg/sql/rowexec/joinreader_test.go b/pkg/sql/rowexec/joinreader_test.go index fdafdb722511..60a386a3f762 100644 --- a/pkg/sql/rowexec/joinreader_test.go +++ b/pkg/sql/rowexec/joinreader_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" @@ -1314,6 +1315,8 @@ func TestIndexJoiner(t *testing.T) { c.outputTypes, c.expected, txn, + s.Stopper(), + s.DistSenderI().(*kvcoord.DistSender), ) }) } diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 0c045c6dbe14..16e7c83afacc 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -148,7 +148,7 @@ func NewProcessor( if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { return nil, err } - if len(core.JoinReader.LookupColumns) == 0 && core.JoinReader.LookupExpr.Empty() { + if core.JoinReader.IsIndexJoin() { return newJoinReader( flowCtx, processorID, core.JoinReader, inputs[0], post, outputs[0], indexJoinReaderType) } diff --git a/pkg/sql/rowexec/project_set_test.go b/pkg/sql/rowexec/project_set_test.go index ae695d067a11..bb02898a91f2 100644 --- a/pkg/sql/rowexec/project_set_test.go +++ b/pkg/sql/rowexec/project_set_test.go @@ -116,6 +116,8 @@ func TestProjectSet(t *testing.T) { append(c.inputTypes, c.spec.GeneratedColumns...), /* outputTypes */ c.expected, nil, + nil, + nil, ) }) } diff --git a/pkg/sql/rowexec/utils_test.go b/pkg/sql/rowexec/utils_test.go index 45a7a5034dae..fffa1d97d290 100644 --- a/pkg/sql/rowexec/utils_test.go +++ b/pkg/sql/rowexec/utils_test.go @@ -16,6 +16,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -25,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/stop" ) // runProcessorTest instantiates a processor with the provided spec, runs it @@ -38,6 +40,8 @@ func runProcessorTest( outputTypes []*types.T, expected rowenc.EncDatumRows, txn *kv.Txn, + stopper *stop.Stopper, + distSender *kvcoord.DistSender, ) { in := distsqlutils.NewRowBuffer(inputTypes, inputRows, distsqlutils.RowBufferArgs{}) out := &distsqlutils.RowBuffer{} @@ -46,7 +50,7 @@ func runProcessorTest( evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(context.Background()) flowCtx := execinfra.FlowCtx{ - Cfg: &execinfra.ServerConfig{Settings: st}, + Cfg: &execinfra.ServerConfig{Settings: st, Stopper: stopper, DistSender: distSender}, EvalCtx: &evalCtx, Txn: txn, } diff --git a/pkg/sql/show_create_clauses.go b/pkg/sql/show_create_clauses.go index d24197404d25..b9d8d7782ba4 100644 --- a/pkg/sql/show_create_clauses.go +++ b/pkg/sql/show_create_clauses.go @@ -563,6 +563,9 @@ func showConstraintClause( f *tree.FmtCtx, ) error { for _, e := range desc.AllActiveAndInactiveChecks() { + if e.Hidden { + continue + } f.WriteString(",\n\t") if len(e.Name) > 0 { f.WriteString("CONSTRAINT ") diff --git a/pkg/sql/show_test.go b/pkg/sql/show_test.go index 0c37b98d6423..979b9e392d95 100644 --- a/pkg/sql/show_test.go +++ b/pkg/sql/show_test.go @@ -270,8 +270,7 @@ func TestShowCreateTable(t *testing.T) { rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT %[1]s_pkey PRIMARY KEY (rowid ASC), INDEX t12_a_idx (a ASC) USING HASH WITH BUCKET_COUNT = 8, - FAMILY "primary" (a, rowid), - CONSTRAINT check_crdb_internal_a_shard_8 CHECK (crdb_internal_a_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)) + FAMILY "primary" (a, rowid) )`, }, } diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index c9bf35ddd16a..d4f5da4aa083 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -191,21 +191,6 @@ func TestSystemTableLiterals(t *testing.T) { } require.NoError(t, catalog.ValidateSelf(gen)) - // TODO (Chengxiong) : remove this check after fixing #68031 - // These two system tables were created before we make shard column as - // virtual columns. We want to keep the hardcoded table descriptors to - // avoid system table migrations. However, in this test we run the `create - // table` statement and compare the result with the hardcoded descriptor, - // and there is discrepancy for sure. So we change the string statement to - // declare the shard column and constraint for it explicitly. The problem - // is that we only set `Hidden=true` when creating a shard column - // internally. User declared constraints has everything the same but with - // `Hidden=false`. So overriding the value here for now. Will remove it - // once we have better logic creating constraints. - if name == "statement_statistics" || name == "transaction_statistics" { - gen.TableDesc().Checks[0].Hidden = true - } - if test.pkg.TableDesc().Equal(gen.TableDesc()) { return }