From 053d361e950ea9cc9d05ac37442fc0628544bd4f Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Fri, 17 Dec 2021 23:54:25 -0500 Subject: [PATCH 1/6] sql: check equivalent constraint when creating hash index Fixes #68031 Previously we only try to create constraint for shard column if it's newly created. We check duplicate constraint for shard column when `Alter Primary Key` and `Create Index`, however the check is simply a name check. This pr adds logic to check equivalent constraint by checking if the formatted expression string is the same. With this logic we can try to create the constraint no matter if a shard column is newly created or not. With this fix, we also don't need to expose the constraint through `SHOW CREATE TABLE` result since we make sure the constraint is created or skipped if one already exists. Release note (sql change): Before this change, the check constraint on shard column used by hash sharded index was printed in the corresponding `SHOW CREATE TABLE`. The constraint had been shown because cockroach lacked logic to ensure that shard columns which are part of hash sharded indexs always have the check constraint which the optimizer relies on to achieve properly optimized plans on hash sharded indexes. We no longer display this constraint in `SHOW CREATE TABLE` as it is now implied by the `USING HASH` clause on the relevant index. --- pkg/sql/alter_primary_key.go | 18 +- pkg/sql/catalog/systemschema/system.go | 10 - pkg/sql/create_index.go | 70 ++++--- pkg/sql/create_table.go | 45 +++-- .../testdata/logic_test/alter_primary_key | 21 +-- .../testdata/logic_test/create_table | 3 +- .../testdata/logic_test/hash_sharded_index | 174 +++++++++++++----- pkg/sql/show_create_clauses.go | 3 + pkg/sql/show_test.go | 3 +- pkg/sql/tests/system_table_test.go | 15 -- 10 files changed, 217 insertions(+), 145 deletions(-) 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/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 a8a49046d010..48393b2265b8 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1485,7 +1485,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 { @@ -1597,7 +1597,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, @@ -1610,18 +1610,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/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/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 } From 5d041510e92d772e510964b1ba88586a5043b463 Mon Sep 17 00:00:00 2001 From: Ben Bardin Date: Wed, 12 Jan 2022 13:15:54 -0500 Subject: [PATCH 2/6] build: Add PATH to .bazelrc for dev builds. Release note: none --- .bazelrc | 2 ++ 1 file changed, 2 insertions(+) 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 From c99b4ecad9ed6c4ae63f58d99102b9014d1a3abf Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 11 Jan 2022 14:10:44 -0800 Subject: [PATCH 3/6] colexec: clean up the usage of the binary overload helper Previously, for projection and selection operators we would always pass in `execgen.OverloadHelper`. Up until recently it served several purposes, but now it is only used in order to pass down the binary function (as well as the eval context) for the cases when we fallback to the row-by-row computation. This commit takes advantage of this observation and cleans up the situation: now the helper is only passed when it is needed which allows us to remove a lot of redundant code. Additionally, the helper itself has been renamed from `OverloadHelper` to `BinaryOverloadHelper`. Release note: None --- .../colexecproj/proj_const_left_ops.eg.go | 2655 ++++-------- .../colexecproj/proj_const_ops_tmpl.go | 25 +- .../colexecproj/proj_const_right_ops.eg.go | 3771 ++++------------- .../colexec/colexecproj/proj_like_ops.eg.go | 48 - .../colexecproj/proj_non_const_ops.eg.go | 3335 +++------------ .../colexecproj/proj_non_const_ops_tmpl.go | 38 +- pkg/sql/colexec/colexecsel/sel_like_ops.eg.go | 48 - .../colexec/colexecsel/selection_ops.eg.go | 2241 +--------- .../colexec/colexecsel/selection_ops_tmpl.go | 21 +- .../colexecwindow/range_offset_handler.eg.go | 623 +-- .../range_offset_handler_tmpl.go | 24 +- pkg/sql/colexec/execgen/BUILD.bazel | 2 +- .../execgen/cmd/execgen/overloads_base.go | 8 +- ...verloads_util.go => overloads_bin_util.go} | 8 +- 14 files changed, 2447 insertions(+), 10400 deletions(-) rename pkg/sql/colexec/execgen/{overloads_util.go => overloads_bin_util.go} (73%) 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 } From a97333dfdb2ece5bc05086af9bf8543ed115e94e Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 30 Dec 2021 10:20:39 -0800 Subject: [PATCH 4/6] execinfrapb: add a helper for index joins based on the JoinReaderSpec Release note: None --- pkg/sql/colexec/colbuilder/execplan.go | 4 ++-- pkg/sql/execinfrapb/processors.go | 6 ++++++ pkg/sql/rowexec/processors.go | 2 +- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index c3835f5c7b62..8264d5d2c8d0 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 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/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) } From ae1236745a54d39021ef3ca650c82f72fae04b6f Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 30 Dec 2021 12:29:26 -0800 Subject: [PATCH 5/6] rowexec: refactor the joinReader to not exceed the batch size The joinReader operates by buffering the input rows until a certain size limit (which is dependent on the strategy). Previously, the buffering would stop right after the size limit is reached or exceeded, and this commit refactors the code to not exceed the limit except in a case of a single large row. This is what we already do for vectorized index joins. Release note: None --- pkg/sql/rowexec/joinreader.go | 69 +++++++++++++++++++++++++---------- 1 file changed, 49 insertions(+), 20 deletions(-) diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index 6fd2a8430365..c0cf9a538b5c 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -145,6 +145,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 @@ -704,28 +709,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 +764,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 { From a4cb6224990a7366df17d2c582c9731383f11206 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 7 Dec 2021 14:43:31 -0800 Subject: [PATCH 6/6] sql,kv: introduce Streamer API and use it for index joins in some cases This commit introduces the Streamer API (see https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20210617_index_lookups_memory_limits.md) as well as its implementation for the simplest case - when requests are unique and can be served in any order. It additionally hooks up the implementation to be used by the index joins in both execution engines. There are three main pieces that this commit adds: 1. the `Streamer` struct itself. It is largely the same as described in the RFC. Some notable changes are: - `Cancel` is renamed to `Close` and is made blocking to ensure that all goroutines kicked off by the `Streamer` exit before `Close` returns. - `Shrink` has been removed from the `budget` struct (see below for more details). - furthermore, the `budget` interface has been unexported and the `streamer` has been tightly coupled with the `budget`'s implementation. - the TODO about collecting DistSQL metadata is removed because we are collecting the LeafTxnFinalState already when using the LeafTxn. 2. the limited `Streamer` implementation - only `OutOfOrder` mode is supported when the requests are unique. Notably, buffering and caching of the results is not implemented yet. 3. `TxnKVStreamer` component that sits below the SQL fetchers, uses the `Streamer`, and is an adapter from `BatchResponse`s to key/value pairs that fetchers understand. Although not used at the moment, `TxnKVStreamer` is written under the assumption that a single result can satisfy multiple requests. The memory budget of the `Streamer` is utilized lazily. The RFC was calling for making a large reservation upfront and then shrinking the budget if we see that we don't need that large reservation; however, while working on this I realized that lazy reservations are a better fit for this. The `Streamer` can reserve up to the specified limit (determined by `distsql_workmem` variable) against the root monitor (in the degenerate case of a single large row more memory will be reserved). The reservation then never shrinks under the assumption that if the reservation has gotten large, it means it was needed for higher concurrency (or large responses), and it is likely to be needed for the same reasons in the future. The layout of the main components of the `Streamer` implementation: - in `Enqueue` we have a logic similar to what DistSender does in order to split each request (that might span multiple ranges) into single-range requests. Those sub-requests are batched together to be evaluated by a single `BatchRequest`. - `workerCoordinator.mainLoop` is responsible for taking single-range batches, estimating the corresponding response size, and issuing requests to be evaluated in parallel while adhering to the provided memory budget. - `workerCoordinator.performRequestAsync` is responsible for populating the `BatchRequest` and then processing the results while updating the memory budget. Current known limitations that will be addressed in the follow-up work: - at the moment a single row can be split across multiple BatchResponses when TargetBytes limit is reached when the table has multiple column families; therefore, we use the streamer only for single column family cases. We will expand the KV API shortly to not split the rows in multiple column family cases. - manual refresh of spans when `ReadWithinUncertaintyIntervalError` is encountered by a single streamer in a single flow is not implemented. It is an optimization that is considered a must for the final implementation in order to not regress on simple cases in terms of retriable errors. This will be implemented shortly as a follow-up. - I'm thinking that eventually we probably want to disable the batch splitting done by the DistSender to eliminate unnecessary blocking when the streamer's splitting was incorrect. This would give us some performance improvements in face of range boundary changes, but it doesn't seem important enough for the initial implementation. Release note: None --- pkg/BUILD.bazel | 1 + pkg/kv/kvclient/kvcoord/batch.go | 12 +- pkg/kv/kvclient/kvcoord/batch_test.go | 2 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 4 +- pkg/kv/kvclient/kvcoord/truncate_test.go | 2 +- pkg/kv/kvclient/kvstreamer/BUILD.bazel | 56 + .../kvstreamer/avg_response_estimator.go | 42 + .../kvstreamer/avg_response_estimator_test.go | 56 + pkg/kv/kvclient/kvstreamer/budget.go | 124 ++ pkg/kv/kvclient/kvstreamer/main_test.go | 31 + pkg/kv/kvclient/kvstreamer/streamer.go | 1165 +++++++++++++++++ pkg/kv/kvclient/kvstreamer/streamer_test.go | 271 ++++ pkg/server/server_sql.go | 1 + pkg/sql/colexec/colbuilder/execplan.go | 16 +- pkg/sql/colfetcher/BUILD.bazel | 1 + pkg/sql/colfetcher/cfetcher.go | 39 +- pkg/sql/colfetcher/index_join.go | 96 +- pkg/sql/distsql/server.go | 13 +- pkg/sql/distsql_running.go | 10 + pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/server_config.go | 3 + pkg/sql/row/BUILD.bazel | 4 + pkg/sql/row/kv_batch_streamer.go | 213 +++ pkg/sql/row/kv_fetcher.go | 11 +- pkg/sql/rowexec/BUILD.bazel | 3 + pkg/sql/rowexec/joinreader.go | 109 +- pkg/sql/rowexec/joinreader_test.go | 3 + pkg/sql/rowexec/project_set_test.go | 2 + pkg/sql/rowexec/utils_test.go | 6 +- 29 files changed, 2251 insertions(+), 46 deletions(-) create mode 100644 pkg/kv/kvclient/kvstreamer/BUILD.bazel create mode 100644 pkg/kv/kvclient/kvstreamer/avg_response_estimator.go create mode 100644 pkg/kv/kvclient/kvstreamer/avg_response_estimator_test.go create mode 100644 pkg/kv/kvclient/kvstreamer/budget.go create mode 100644 pkg/kv/kvclient/kvstreamer/main_test.go create mode 100644 pkg/kv/kvclient/kvstreamer/streamer.go create mode 100644 pkg/kv/kvclient/kvstreamer/streamer_test.go create mode 100644 pkg/sql/row/kv_batch_streamer.go 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/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 8264d5d2c8d0..ae37e3eb09ff 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -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/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/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/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 c0cf9a538b5c..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 @@ -285,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, @@ -293,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 { @@ -443,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 } @@ -839,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() } @@ -971,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 } @@ -985,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/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, }