From 8534ad1ae5c4b7ba10a47d51a6ad2ab5887d6021 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 5 Aug 2020 17:53:40 -0700 Subject: [PATCH 1/3] col*: introduce the concept of capacity to coldata.Batch This commit introduces the concept of capacity to `coldata.Batch` which describes the maximum number of tuples the batch can store. Note that it is a lower bound meaning that some vectors in the batch might have larger underlying capacity (e.g. when they were appended to). Additionally this commit does several mechanical changes to rename the methods. Release note: None --- pkg/ccl/importccl/read_import_workload.go | 2 +- pkg/ccl/workloadccl/allccl/all_test.go | 2 +- pkg/col/coldata/batch.go | 87 ++++++++++++------- pkg/col/coldata/batch_test.go | 8 +- pkg/col/coldata/vec.go | 33 ++++--- pkg/col/coldatatestutils/random_testutils.go | 2 +- pkg/col/coldatatestutils/utils.go | 2 +- pkg/col/colserde/arrowbatchconverter_test.go | 6 +- pkg/col/colserde/file_test.go | 8 +- pkg/sql/colcontainer/partitionedqueue_test.go | 4 +- pkg/sql/colencoding/value_encoding_test.go | 2 +- pkg/sql/colexec/and_or_projection_test.go | 2 +- pkg/sql/colexec/builtin_funcs_test.go | 4 +- pkg/sql/colexec/cancel_checker_test.go | 2 +- pkg/sql/colexec/columnarizer.go | 2 +- pkg/sql/colexec/count.go | 4 +- pkg/sql/colexec/deselector.go | 2 +- pkg/sql/colexec/deselector_test.go | 2 +- pkg/sql/colexec/distinct_test.go | 2 +- pkg/sql/colexec/external_hash_joiner.go | 8 +- pkg/sql/colexec/external_hash_joiner_test.go | 4 +- pkg/sql/colexec/external_sort_test.go | 2 +- pkg/sql/colexec/hash_aggregator.go | 2 +- pkg/sql/colexec/hashjoiner.go | 6 +- pkg/sql/colexec/hashjoiner_test.go | 2 +- pkg/sql/colexec/hashtable.go | 2 +- pkg/sql/colexec/like_ops_test.go | 2 +- pkg/sql/colexec/materializer_test.go | 2 +- pkg/sql/colexec/mergejoiner.go | 35 +++----- pkg/sql/colexec/mergejoiner_test.go | 2 +- pkg/sql/colexec/offset_test.go | 2 +- pkg/sql/colexec/operator.go | 2 +- pkg/sql/colexec/ordered_aggregator.go | 4 +- pkg/sql/colexec/ordered_synchronizer.eg.go | 2 +- pkg/sql/colexec/ordered_synchronizer_test.go | 2 +- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 2 +- pkg/sql/colexec/ordinality_test.go | 2 +- .../parallel_unordered_synchronizer_test.go | 2 +- pkg/sql/colexec/projection_ops_test.go | 4 +- pkg/sql/colexec/relative_rank.eg.go | 40 ++++----- pkg/sql/colexec/relative_rank_tmpl.go | 12 +-- pkg/sql/colexec/routers.go | 2 +- pkg/sql/colexec/routers_test.go | 4 +- pkg/sql/colexec/select_in_test.go | 2 +- pkg/sql/colexec/selection_ops_test.go | 4 +- pkg/sql/colexec/simple_project_test.go | 2 +- pkg/sql/colexec/sort.go | 8 +- pkg/sql/colexec/sort_chunks.go | 4 +- pkg/sql/colexec/sort_chunks_test.go | 2 +- pkg/sql/colexec/sort_test.go | 4 +- pkg/sql/colexec/sorttopk.go | 6 +- pkg/sql/colexec/spilling_queue.go | 2 +- pkg/sql/colexec/stats_test.go | 2 +- pkg/sql/colexec/types_integration_test.go | 2 +- pkg/sql/colexec/unordered_distinct.go | 2 +- pkg/sql/colexec/utils.go | 6 +- pkg/sql/colexec/utils_test.go | 10 +-- pkg/sql/colexecbase/testutils.go | 2 +- pkg/sql/colfetcher/cfetcher.go | 2 +- pkg/sql/colflow/colrpc/colrpc_test.go | 6 +- pkg/sql/colflow/colrpc/inbox.go | 2 +- pkg/sql/colflow/colrpc/outbox_test.go | 2 +- .../colflow/vectorized_flow_shutdown_test.go | 2 +- pkg/sql/colflow/vectorized_flow_space_test.go | 2 +- pkg/sql/colmem/allocator.go | 37 ++++---- pkg/testutils/lint/lint_test.go | 4 +- pkg/workload/csv.go | 4 +- pkg/workload/workload.go | 2 +- 68 files changed, 227 insertions(+), 217 deletions(-) diff --git a/pkg/ccl/importccl/read_import_workload.go b/pkg/ccl/importccl/read_import_workload.go index 6006fd1ccb67..a022c68e8092 100644 --- a/pkg/ccl/importccl/read_import_workload.go +++ b/pkg/ccl/importccl/read_import_workload.go @@ -229,7 +229,7 @@ func (w *WorkloadKVConverter) Worker(ctx context.Context, evalCtx *tree.EvalCont } var alloc sqlbase.DatumAlloc var a bufalloc.ByteAllocator - cb := coldata.NewMemBatchWithSize(nil /* types */, 0 /* size */, coldata.StandardColumnFactory) + cb := coldata.NewMemBatchWithCapacity(nil /* typs */, 0 /* capacity */, coldata.StandardColumnFactory) for { batchIdx := int(atomic.AddInt64(&w.batchIdxAtomic, 1)) diff --git a/pkg/ccl/workloadccl/allccl/all_test.go b/pkg/ccl/workloadccl/allccl/all_test.go index 4390d4cb3e3d..25262347821d 100644 --- a/pkg/ccl/workloadccl/allccl/all_test.go +++ b/pkg/ccl/workloadccl/allccl/all_test.go @@ -195,7 +195,7 @@ func hashTableInitialData( h hash.Hash, data workload.BatchedTuples, a *bufalloc.ByteAllocator, ) error { var scratch [8]byte - b := coldata.NewMemBatchWithSize(nil /* types */, 0 /* size */, coldata.StandardColumnFactory) + b := coldata.NewMemBatchWithCapacity(nil /* typs */, 0 /* capacity */, coldata.StandardColumnFactory) for batchIdx := 0; batchIdx < data.NumBatches; batchIdx++ { *a = (*a)[:0] data.FillBatch(batchIdx, b, a) diff --git a/pkg/col/coldata/batch.go b/pkg/col/coldata/batch.go index 5d2b63a77667..57f58ae83937 100644 --- a/pkg/col/coldata/batch.go +++ b/pkg/col/coldata/batch.go @@ -29,6 +29,11 @@ type Batch interface { Length() int // SetLength sets the number of values in the columns in the batch. SetLength(int) + // Capacity returns the maximum number of values that can be stored in the + // columns in the batch. Note that it could be a lower bound meaning some + // of the Vecs could actually have larger underlying capacity (for example, + // if they have been appended to). + Capacity() int // Width returns the number of columns in the batch. Width() int // ColVec returns the ith Vec in this batch. @@ -96,19 +101,18 @@ func ResetBatchSizeForTests() { atomic.SwapInt64(&batchSize, defaultBatchSize) } -// NewMemBatch allocates a new in-memory Batch. An unsupported type will create -// a placeholder Vec that may not be accessed. +// NewMemBatch allocates a new in-memory Batch. // TODO(jordan): pool these allocations. func NewMemBatch(typs []*types.T, factory ColumnFactory) Batch { - return NewMemBatchWithSize(typs, BatchSize(), factory) + return NewMemBatchWithCapacity(typs, BatchSize(), factory) } -// NewMemBatchWithSize allocates a new in-memory Batch with the given column -// size. Use for operators that have a precisely-sized output batch. -func NewMemBatchWithSize(typs []*types.T, size int, factory ColumnFactory) Batch { - b := NewMemBatchNoCols(typs, size).(*MemBatch) +// NewMemBatchWithCapacity allocates a new in-memory Batch with the given +// column size. Use for operators that have a precisely-sized output batch. +func NewMemBatchWithCapacity(typs []*types.T, capacity int, factory ColumnFactory) Batch { + b := NewMemBatchNoCols(typs, capacity).(*MemBatch) for i, t := range typs { - b.b[i] = NewMemColumn(t, size, factory) + b.b[i] = NewMemColumn(t, capacity, factory) if b.b[i].CanonicalTypeFamily() == types.BytesFamily { b.bytesVecIdxs = append(b.bytesVecIdxs, i) } @@ -119,20 +123,21 @@ func NewMemBatchWithSize(typs []*types.T, size int, factory ColumnFactory) Batch // NewMemBatchNoCols creates a "skeleton" of new in-memory Batch. It allocates // memory for the selection vector but does *not* allocate any memory for the // column vectors - those will have to be added separately. -func NewMemBatchNoCols(typs []*types.T, size int) Batch { - if max := math.MaxUint16; size > max { - panic(fmt.Sprintf(`batches cannot have length larger than %d; requested %d`, max, size)) +func NewMemBatchNoCols(typs []*types.T, capacity int) Batch { + if max := math.MaxUint16; capacity > max { + panic(fmt.Sprintf(`batches cannot have capacity larger than %d; requested %d`, max, capacity)) } b := &MemBatch{} + b.capacity = capacity b.b = make([]Vec, len(typs)) - b.sel = make([]int, size) + b.sel = make([]int, capacity) return b } // ZeroBatch is a schema-less Batch of length 0. var ZeroBatch = &zeroBatch{ - MemBatch: NewMemBatchWithSize( - nil /* types */, 0 /* size */, StandardColumnFactory, + MemBatch: NewMemBatchWithCapacity( + nil /* typs */, 0 /* capacity */, StandardColumnFactory, ).(*MemBatch), } @@ -148,6 +153,10 @@ func (b *zeroBatch) Length() int { return 0 } +func (b *zeroBatch) Capacity() int { + return 0 +} + func (b *zeroBatch) SetLength(int) { panic("length should not be changed on zero batch") } @@ -170,9 +179,12 @@ func (b *zeroBatch) Reset([]*types.T, int, ColumnFactory) { // MemBatch is an in-memory implementation of Batch. type MemBatch struct { - // length of batch or sel in tuples - n int - // slice of columns in this batch. + // length is the length of batch or sel in tuples. + length int + // capacity is the maximum number of tuples that can be stored in this + // MemBatch. + capacity int + // b is the slice of columns in this batch. b []Vec // bytesVecIdxs stores the indices of all vectors of Bytes type in b. Bytes // vectors require special handling, so rather than iterating over all @@ -180,14 +192,21 @@ type MemBatch struct { // separately. bytesVecIdxs []int useSel bool - // if useSel is true, a selection vector from upstream. a selection vector is - // a list of selected column indexes in this memBatch's columns. + // sel is - if useSel is true - a selection vector from upstream. A + // selection vector is a list of selected tuple indices in this memBatch's + // columns (tuples for which indices are not in sel are considered to be + // "not present"). sel []int } // Length implements the Batch interface. func (m *MemBatch) Length() int { - return m.n + return m.length +} + +// Capacity implements the Batch interface. +func (m *MemBatch) Capacity() int { + return m.capacity } // Width implements the Batch interface. @@ -219,11 +238,11 @@ func (m *MemBatch) SetSelection(b bool) { } // SetLength implements the Batch interface. -func (m *MemBatch) SetLength(n int) { - m.n = n - if n > 0 { +func (m *MemBatch) SetLength(length int) { + m.length = length + if length > 0 { for _, bytesVecIdx := range m.bytesVecIdxs { - m.b[bytesVecIdx].Bytes().UpdateOffsetsToBeNonDecreasing(n) + m.b[bytesVecIdx].Bytes().UpdateOffsetsToBeNonDecreasing(length) } } } @@ -247,11 +266,7 @@ func (m *MemBatch) ReplaceCol(col Vec, colIdx int) { // Reset implements the Batch interface. func (m *MemBatch) Reset(typs []*types.T, length int, factory ColumnFactory) { - // The columns are always sized the same as the selection vector, so use it as - // a shortcut for the capacity (like a go slice, the batch's `Length` could be - // shorter than the capacity). We could be more defensive and type switch - // every column to verify its capacity, but that doesn't seem necessary yet. - cannotReuse := m == nil || len(m.sel) < length || m.Width() < len(typs) + cannotReuse := m == nil || m.Capacity() < length || m.Width() < len(typs) for i := 0; i < len(typs) && !cannotReuse; i++ { // TODO(yuzefovich): change this when DatumVec is introduced. // TODO(yuzefovich): requiring that types are "identical" might be an @@ -259,18 +274,26 @@ func (m *MemBatch) Reset(typs []*types.T, length int, factory ColumnFactory) { // but non-identical types. Think through this more. if !m.ColVec(i).Type().Identical(typs[i]) { cannotReuse = true + break } } if cannotReuse { - *m = *NewMemBatchWithSize(typs, length, factory).(*MemBatch) + *m = *NewMemBatchWithCapacity(typs, length, factory).(*MemBatch) m.SetLength(length) return } // Yay! We can reuse m. NB It's not specified in the Reset contract, but // probably a good idea to keep all modifications below this line. - m.SetLength(length) - m.sel = m.sel[:length] + // + // Note that we're intentionally not calling m.SetLength() here because + // that would update offsets in the bytes vectors which is not necessary + // since those will get reset in ResetInternalBatch anyway. + m.length = length m.b = m.b[:len(typs)] + for i := range m.b { + m.b[i].SetLength(length) + } + m.sel = m.sel[:length] for i, idx := range m.bytesVecIdxs { if idx >= len(typs) { m.bytesVecIdxs = m.bytesVecIdxs[:i] diff --git a/pkg/col/coldata/batch_test.go b/pkg/col/coldata/batch_test.go index 44687bf69daf..131b8fdaab04 100644 --- a/pkg/col/coldata/batch_test.go +++ b/pkg/col/coldata/batch_test.go @@ -86,21 +86,21 @@ func TestBatchReset(t *testing.T) { resetAndCheck(b, typsInt, 1, true) // Exact length, reuse - b = coldata.NewMemBatchWithSize(typsInt, 1, coldata.StandardColumnFactory) + b = coldata.NewMemBatchWithCapacity(typsInt, 1, coldata.StandardColumnFactory) resetAndCheck(b, typsInt, 1, true) // Insufficient capacity, don't reuse - b = coldata.NewMemBatchWithSize(typsInt, 1, coldata.StandardColumnFactory) + b = coldata.NewMemBatchWithCapacity(typsInt, 1, coldata.StandardColumnFactory) resetAndCheck(b, typsInt, 2, false) // Selection vector gets reset - b = coldata.NewMemBatchWithSize(typsInt, 1, coldata.StandardColumnFactory) + b = coldata.NewMemBatchWithCapacity(typsInt, 1, coldata.StandardColumnFactory) b.SetSelection(true) b.Selection()[0] = 7 resetAndCheck(b, typsInt, 1, true) // Nulls gets reset - b = coldata.NewMemBatchWithSize(typsInt, 1, coldata.StandardColumnFactory) + b = coldata.NewMemBatchWithCapacity(typsInt, 1, coldata.StandardColumnFactory) b.ColVec(0).Nulls().SetNull(0) resetAndCheck(b, typsInt, 1, true) diff --git a/pkg/col/coldata/vec.go b/pkg/col/coldata/vec.go index 9fd9d6aa0d6b..46695efd1f2a 100644 --- a/pkg/col/coldata/vec.go +++ b/pkg/col/coldata/vec.go @@ -131,8 +131,7 @@ type Vec interface { // Capacity returns the capacity of the Golang's slice that is underlying // this Vec. Note that if there is no "slice" (like in case of flat bytes), - // the "capacity" of such object is undefined, so is the behavior of this - // method. + // then "capacity" of such object is equal to the number of elements. Capacity() int } @@ -149,7 +148,7 @@ type memColumn struct { // ColumnFactory is an interface that can construct columns for Batches. type ColumnFactory interface { - MakeColumn(t *types.T, n int) Column + MakeColumn(t *types.T, length int) Column } type defaultColumnFactory struct{} @@ -158,31 +157,31 @@ type defaultColumnFactory struct{} // explicitly supported by the vectorized engine (i.e. not datum-backed). var StandardColumnFactory ColumnFactory = &defaultColumnFactory{} -func (cf *defaultColumnFactory) MakeColumn(t *types.T, n int) Column { +func (cf *defaultColumnFactory) MakeColumn(t *types.T, length int) Column { switch canonicalTypeFamily := typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()); canonicalTypeFamily { case types.BoolFamily: - return make(Bools, n) + return make(Bools, length) case types.BytesFamily: - return NewBytes(n) + return NewBytes(length) case types.IntFamily: switch t.Width() { case 16: - return make(Int16s, n) + return make(Int16s, length) case 32: - return make(Int32s, n) + return make(Int32s, length) case 0, 64: - return make(Int64s, n) + return make(Int64s, length) default: panic(fmt.Sprintf("unexpected integer width: %d", t.Width())) } case types.FloatFamily: - return make(Float64s, n) + return make(Float64s, length) case types.DecimalFamily: - return make(Decimals, n) + return make(Decimals, length) case types.TimestampTZFamily: - return make(Times, n) + return make(Times, length) case types.IntervalFamily: - return make(Durations, n) + return make(Durations, length) default: panic(fmt.Sprintf("StandardColumnFactory doesn't support %s", t)) } @@ -190,12 +189,12 @@ func (cf *defaultColumnFactory) MakeColumn(t *types.T, n int) Column { // NewMemColumn returns a new memColumn, initialized with a length using the // given column factory. -func NewMemColumn(t *types.T, n int, factory ColumnFactory) Vec { +func NewMemColumn(t *types.T, length int, factory ColumnFactory) Vec { return &memColumn{ t: t, canonicalTypeFamily: typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()), - col: factory.MakeColumn(t, n), - nulls: NewNulls(n), + col: factory.MakeColumn(t, length), + nulls: NewNulls(length), } } @@ -340,7 +339,7 @@ func (m *memColumn) Capacity() int { case types.BoolFamily: return cap(m.col.(Bools)) case types.BytesFamily: - panic("Capacity should not be called on Vec of Bytes type") + return m.Bytes().Len() case types.IntFamily: switch m.t.Width() { case 16: diff --git a/pkg/col/coldatatestutils/random_testutils.go b/pkg/col/coldatatestutils/random_testutils.go index 5c9f45b33699..35728a30dab6 100644 --- a/pkg/col/coldatatestutils/random_testutils.go +++ b/pkg/col/coldatatestutils/random_testutils.go @@ -204,7 +204,7 @@ func RandomBatch( length int, nullProbability float64, ) coldata.Batch { - batch := allocator.NewMemBatchWithSize(typs, capacity) + batch := allocator.NewMemBatchWithFixedCapacity(typs, capacity) if length == 0 { length = capacity } diff --git a/pkg/col/coldatatestutils/utils.go b/pkg/col/coldatatestutils/utils.go index c2edd135202d..0f39ebf24d6a 100644 --- a/pkg/col/coldatatestutils/utils.go +++ b/pkg/col/coldatatestutils/utils.go @@ -22,7 +22,7 @@ import ( func CopyBatch( original coldata.Batch, typs []*types.T, factory coldata.ColumnFactory, ) coldata.Batch { - b := coldata.NewMemBatchWithSize(typs, original.Length(), factory) + b := coldata.NewMemBatchWithCapacity(typs, original.Length(), factory) b.SetLength(original.Length()) for colIdx, col := range original.ColVecs() { b.ColVec(colIdx).Copy(coldata.CopySliceArgs{ diff --git a/pkg/col/colserde/arrowbatchconverter_test.go b/pkg/col/colserde/arrowbatchconverter_test.go index 7865a338659d..d2da231cbb79 100644 --- a/pkg/col/colserde/arrowbatchconverter_test.go +++ b/pkg/col/colserde/arrowbatchconverter_test.go @@ -55,7 +55,7 @@ func TestArrowBatchConverterRandom(t *testing.T) { arrowData, err := c.BatchToArrow(b) require.NoError(t, err) - actual := testAllocator.NewMemBatchWithSize(typs, b.Length()) + actual := testAllocator.NewMemBatchWithFixedCapacity(typs, b.Length()) require.NoError(t, c.ArrowToBatch(arrowData, actual)) coldata.AssertEquivalentBatches(t, expected, actual) @@ -85,7 +85,7 @@ func roundTripBatch( if err := r.Deserialize(&arrowDataOut, buf.Bytes()); err != nil { return nil, err } - actual := testAllocator.NewMemBatchWithSize(typs, b.Length()) + actual := testAllocator.NewMemBatchWithFixedCapacity(typs, b.Length()) if err := c.ArrowToBatch(arrowDataOut, actual); err != nil { return nil, err } @@ -204,7 +204,7 @@ func BenchmarkArrowBatchConverter(b *testing.B) { data, err := c.BatchToArrow(batch) require.NoError(b, err) testPrefix := fmt.Sprintf("%s/nullFraction=%0.2f", typ.String(), nullFraction) - result := testAllocator.NewMemBatch([]*types.T{typ}) + result := testAllocator.NewMemBatchWithMaxCapacity([]*types.T{typ}) b.Run(testPrefix+"/ArrowToBatch", func(b *testing.B) { b.SetBytes(numBytes[typIdx]) for i := 0; i < b.N; i++ { diff --git a/pkg/col/colserde/file_test.go b/pkg/col/colserde/file_test.go index 5ce3699a9cb5..bcecc2f6792d 100644 --- a/pkg/col/colserde/file_test.go +++ b/pkg/col/colserde/file_test.go @@ -45,7 +45,7 @@ func TestFileRoundtrip(t *testing.T) { // buffer. for i := 0; i < 2; i++ { func() { - roundtrip := testAllocator.NewMemBatchWithSize(typs, b.Length()) + roundtrip := testAllocator.NewMemBatchWithFixedCapacity(typs, b.Length()) d, err := colserde.NewFileDeserializerFromBytes(typs, buf.Bytes()) require.NoError(t, err) defer func() { require.NoError(t, d.Close()) }() @@ -81,7 +81,7 @@ func TestFileRoundtrip(t *testing.T) { // file. for i := 0; i < 2; i++ { func() { - roundtrip := testAllocator.NewMemBatchWithSize(typs, b.Length()) + roundtrip := testAllocator.NewMemBatchWithFixedCapacity(typs, b.Length()) d, err := colserde.NewFileDeserializerFromPath(typs, path) require.NoError(t, err) defer func() { require.NoError(t, d.Close()) }() @@ -107,7 +107,7 @@ func TestFileIndexing(t *testing.T) { require.NoError(t, err) for i := 0; i < numInts; i++ { - b := testAllocator.NewMemBatchWithSize(typs, batchSize) + b := testAllocator.NewMemBatchWithFixedCapacity(typs, batchSize) b.SetLength(batchSize) b.ColVec(0).Int64()[0] = int64(i) require.NoError(t, s.AppendBatch(b)) @@ -120,7 +120,7 @@ func TestFileIndexing(t *testing.T) { require.Equal(t, typs, d.Typs()) require.Equal(t, numInts, d.NumBatches()) for batchIdx := numInts - 1; batchIdx >= 0; batchIdx-- { - b := testAllocator.NewMemBatchWithSize(typs, batchSize) + b := testAllocator.NewMemBatchWithFixedCapacity(typs, batchSize) require.NoError(t, d.GetBatch(batchIdx, b)) require.Equal(t, batchSize, b.Length()) require.Equal(t, 1, b.Width()) diff --git a/pkg/sql/colcontainer/partitionedqueue_test.go b/pkg/sql/colcontainer/partitionedqueue_test.go index 616c8fd8638f..50b89648ee08 100644 --- a/pkg/sql/colcontainer/partitionedqueue_test.go +++ b/pkg/sql/colcontainer/partitionedqueue_test.go @@ -94,7 +94,7 @@ func TestPartitionedDiskQueue(t *testing.T) { var ( ctx = context.Background() typs = []*types.T{types.Int} - batch = testAllocator.NewMemBatch(typs) + batch = testAllocator.NewMemBatchWithMaxCapacity(typs) sem = &colexecbase.TestingSemaphore{} ) batch.SetLength(coldata.BatchSize()) @@ -145,7 +145,7 @@ func TestPartitionedDiskQueueSimulatedExternal(t *testing.T) { var ( ctx = context.Background() typs = []*types.T{types.Int} - batch = testAllocator.NewMemBatch(typs) + batch = testAllocator.NewMemBatchWithMaxCapacity(typs) rng, _ = randutil.NewPseudoRand() // maxPartitions is in [1, 10]. The maximum partitions on a single level. maxPartitions = 1 + rng.Intn(10) diff --git a/pkg/sql/colencoding/value_encoding_test.go b/pkg/sql/colencoding/value_encoding_test.go index 6c03b7ed6972..0ba39a914ca0 100644 --- a/pkg/sql/colencoding/value_encoding_test.go +++ b/pkg/sql/colencoding/value_encoding_test.go @@ -45,7 +45,7 @@ func TestDecodeTableValueToCol(t *testing.T) { t.Fatal(err) } } - batch := coldata.NewMemBatchWithSize(typs, 1 /* size */, coldataext.NewExtendedColumnFactory(nil /*evalCtx */)) + batch := coldata.NewMemBatchWithCapacity(typs, 1 /* capacity */, coldataext.NewExtendedColumnFactory(nil /*evalCtx */)) for i := 0; i < nCols; i++ { typeOffset, dataOffset, _, typ, err := encoding.DecodeValueTag(buf) fmt.Println(typ) diff --git a/pkg/sql/colexec/and_or_projection_test.go b/pkg/sql/colexec/and_or_projection_test.go index 6960eabf9e9c..d616d1b41a60 100644 --- a/pkg/sql/colexec/and_or_projection_test.go +++ b/pkg/sql/colexec/and_or_projection_test.go @@ -231,7 +231,7 @@ func benchmarkLogicalProjOp( } rng, _ := randutil.NewPseudoRand() - batch := testAllocator.NewMemBatch([]*types.T{types.Bool, types.Bool}) + batch := testAllocator.NewMemBatchWithMaxCapacity([]*types.T{types.Bool, types.Bool}) col1 := batch.ColVec(0).Bool() col2 := batch.ColVec(0).Bool() for i := 0; i < coldata.BatchSize(); i++ { diff --git a/pkg/sql/colexec/builtin_funcs_test.go b/pkg/sql/colexec/builtin_funcs_test.go index f72911fe2dfc..4c161e75bec0 100644 --- a/pkg/sql/colexec/builtin_funcs_test.go +++ b/pkg/sql/colexec/builtin_funcs_test.go @@ -95,7 +95,7 @@ func benchmarkBuiltinFunctions(b *testing.B, useSelectionVector bool, hasNulls b }, } - batch := testAllocator.NewMemBatch([]*types.T{types.Int}) + batch := testAllocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) col := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { @@ -159,7 +159,7 @@ func BenchmarkCompareSpecializedOperators(b *testing.B) { defer evalCtx.Stop(ctx) typs := []*types.T{types.String, types.Int, types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) outputIdx := 3 bCol := batch.ColVec(0).Bytes() sCol := batch.ColVec(1).Int64() diff --git a/pkg/sql/colexec/cancel_checker_test.go b/pkg/sql/colexec/cancel_checker_test.go index 8aac2ebd8cf0..a8970e22b178 100644 --- a/pkg/sql/colexec/cancel_checker_test.go +++ b/pkg/sql/colexec/cancel_checker_test.go @@ -31,7 +31,7 @@ func TestCancelChecker(t *testing.T) { defer log.Scope(t).Close(t) ctx, cancel := context.WithCancel(context.Background()) typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) op := NewCancelChecker(NewNoop(colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs))) cancel() err := colexecerror.CatchVectorizedRuntimeError(func() { diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index 01b5141739b7..17cae80a27b0 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -81,7 +81,7 @@ func (c *Columnarizer) Init() { // internal objects several times if Init method is called more than once, so // we have this check in place. if c.initStatus == OperatorNotInitialized { - c.batch = c.allocator.NewMemBatch(c.typs) + c.batch = c.allocator.NewMemBatchWithMaxCapacity(c.typs) c.buffered = make(sqlbase.EncDatumRows, coldata.BatchSize()) for i := range c.buffered { c.buffered[i] = make(sqlbase.EncDatumRow, len(c.typs)) diff --git a/pkg/sql/colexec/count.go b/pkg/sql/colexec/count.go index e7747f5f07b0..91ae1f08d1a2 100644 --- a/pkg/sql/colexec/count.go +++ b/pkg/sql/colexec/count.go @@ -38,7 +38,9 @@ func NewCountOp(allocator *colmem.Allocator, input colexecbase.Operator) colexec c := &countOp{ OneInputNode: NewOneInputNode(input), } - c.internalBatch = allocator.NewMemBatchWithSize([]*types.T{types.Int}, 1) + c.internalBatch = allocator.NewMemBatchWithFixedCapacity( + []*types.T{types.Int}, 1, /* capacity */ + ) return c } diff --git a/pkg/sql/colexec/deselector.go b/pkg/sql/colexec/deselector.go index b97f56f2d0e2..bceecbb6d74d 100644 --- a/pkg/sql/colexec/deselector.go +++ b/pkg/sql/colexec/deselector.go @@ -79,7 +79,7 @@ func (p *deselectorOp) Next(ctx context.Context) coldata.Batch { func (p *deselectorOp) resetOutput() { if p.output == nil { - p.output = p.allocator.NewMemBatch(p.inputTypes) + p.output = p.allocator.NewMemBatchWithMaxCapacity(p.inputTypes) } else { p.output.ResetInternalBatch() } diff --git a/pkg/sql/colexec/deselector_test.go b/pkg/sql/colexec/deselector_test.go index 7691a1396509..9ab060b23fb0 100644 --- a/pkg/sql/colexec/deselector_test.go +++ b/pkg/sql/colexec/deselector_test.go @@ -88,7 +88,7 @@ func BenchmarkDeselector(b *testing.B) { inputTypes[colIdx] = types.Int } - batch := testAllocator.NewMemBatch(inputTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(inputTypes) for colIdx := 0; colIdx < nCols; colIdx++ { col := batch.ColVec(colIdx).Int64() diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index 500535647a1a..4799e19ed6fe 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -243,7 +243,7 @@ func BenchmarkDistinct(b *testing.B) { for i := range typs { typs[i] = types.Int } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) distinctCols := []uint32{0, 1, 2, 3}[:nCols] // We have the following equation: diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index 246343e9decc..ca0969aa3024 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -387,8 +387,8 @@ func NewExternalHashJoiner( if ehj.memState.maxRightPartitionSizeToJoin < externalHJMinimalMaxRightPartitionSize { ehj.memState.maxRightPartitionSizeToJoin = externalHJMinimalMaxRightPartitionSize } - ehj.scratch.leftBatch = unlimitedAllocator.NewMemBatch(spec.left.sourceTypes) - ehj.recursiveScratch.leftBatch = unlimitedAllocator.NewMemBatch(spec.left.sourceTypes) + ehj.scratch.leftBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.left.sourceTypes) + ehj.recursiveScratch.leftBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.left.sourceTypes) sameSourcesSchema := len(spec.left.sourceTypes) == len(spec.right.sourceTypes) for i, leftType := range spec.left.sourceTypes { if i < len(spec.right.sourceTypes) && !leftType.Identical(spec.right.sourceTypes[i]) { @@ -401,8 +401,8 @@ func NewExternalHashJoiner( ehj.scratch.rightBatch = ehj.scratch.leftBatch ehj.recursiveScratch.rightBatch = ehj.recursiveScratch.leftBatch } else { - ehj.scratch.rightBatch = unlimitedAllocator.NewMemBatch(spec.right.sourceTypes) - ehj.recursiveScratch.rightBatch = unlimitedAllocator.NewMemBatch(spec.right.sourceTypes) + ehj.scratch.rightBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.right.sourceTypes) + ehj.recursiveScratch.rightBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.right.sourceTypes) } ehj.testingKnobs.numForcedRepartitions = numForcedRepartitions ehj.testingKnobs.delegateFDAcquisitions = delegateFDAcquisitions diff --git a/pkg/sql/colexec/external_hash_joiner_test.go b/pkg/sql/colexec/external_hash_joiner_test.go index c8f57485cbee..750a171a2c09 100644 --- a/pkg/sql/colexec/external_hash_joiner_test.go +++ b/pkg/sql/colexec/external_hash_joiner_test.go @@ -135,7 +135,7 @@ func TestExternalHashJoinerFallbackToSortMergeJoin(t *testing.T) { }, } sourceTypes := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(sourceTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(sourceTypes) // We don't need to set the data since zero values in the columns work. batch.SetLength(coldata.BatchSize()) nBatches := 2 @@ -205,7 +205,7 @@ func BenchmarkExternalHashJoiner(b *testing.B) { sourceTypes[colIdx] = types.Int } - batch := testAllocator.NewMemBatch(sourceTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(sourceTypes) for colIdx := 0; colIdx < nCols; colIdx++ { col := batch.ColVec(colIdx).Int64() for i := 0; i < coldata.BatchSize(); i++ { diff --git a/pkg/sql/colexec/external_sort_test.go b/pkg/sql/colexec/external_sort_test.go index ac7d9dcf5ff5..b96f40389579 100644 --- a/pkg/sql/colexec/external_sort_test.go +++ b/pkg/sql/colexec/external_sort_test.go @@ -261,7 +261,7 @@ func BenchmarkExternalSort(b *testing.B) { for i := range typs { typs[i] = types.Int } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) ordCols := make([]execinfrapb.Ordering_Column, nCols) for i := range ordCols { diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index 4b72d24b06c5..d50a3529b362 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -142,7 +142,7 @@ func NewHashAggregator( func (op *hashAggregator) Init() { op.input.Init() - op.output = op.allocator.NewMemBatch(op.outputTypes) + op.output = op.allocator.NewMemBatchWithMaxCapacity(op.outputTypes) op.scratch.eqChains = make([][]int, coldata.BatchSize()) op.scratch.intSlice = make([]int, coldata.BatchSize()) op.scratch.anotherIntSlice = make([]int, coldata.BatchSize()) diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index e7ca64795212..f8578b82b4fa 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -238,7 +238,9 @@ func (hj *hashJoiner) Init() { probeMode, ) - hj.exportBufferedState.rightWindowedBatch = hj.allocator.NewMemBatchWithSize(hj.spec.right.sourceTypes, 0 /* size */) + hj.exportBufferedState.rightWindowedBatch = hj.allocator.NewMemBatchWithFixedCapacity( + hj.spec.right.sourceTypes, 0, /* size */ + ) hj.state = hjBuilding } @@ -594,7 +596,7 @@ func (hj *hashJoiner) resetOutput() { if hj.spec.joinType.ShouldIncludeRightColsInOutput() { outputTypes = append(outputTypes, hj.spec.right.sourceTypes...) } - hj.output = hj.allocator.NewMemBatch(outputTypes) + hj.output = hj.allocator.NewMemBatchWithMaxCapacity(outputTypes) } else { hj.output.ResetInternalBatch() } diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index f733e24748ef..2010543031f2 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -1035,7 +1035,7 @@ func BenchmarkHashJoiner(b *testing.B) { sourceTypes[colIdx] = types.Int } - batch := testAllocator.NewMemBatch(sourceTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(sourceTypes) for colIdx := 0; colIdx < nCols; colIdx++ { col := batch.ColVec(colIdx).Int64() diff --git a/pkg/sql/colexec/hashtable.go b/pkg/sql/colexec/hashtable.go index ac688e05708d..b259e347fef0 100644 --- a/pkg/sql/colexec/hashtable.go +++ b/pkg/sql/colexec/hashtable.go @@ -207,7 +207,7 @@ func newHashTable( differs: make([]bool, coldata.BatchSize()), }, - vals: newAppendOnlyBufferedBatch(allocator, sourceTypes, 0 /* initialSize */), + vals: newAppendOnlyBufferedBatch(allocator, sourceTypes), keyCols: eqCols, numBuckets: numBuckets, allowNullEquality: allowNullEquality, diff --git a/pkg/sql/colexec/like_ops_test.go b/pkg/sql/colexec/like_ops_test.go index 16ceef4db202..ca44cd5a77e5 100644 --- a/pkg/sql/colexec/like_ops_test.go +++ b/pkg/sql/colexec/like_ops_test.go @@ -105,7 +105,7 @@ func BenchmarkLikeOps(b *testing.B) { ctx := context.Background() typs := []*types.T{types.Bytes} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) col := batch.ColVec(0).Bytes() width := 64 for i := 0; i < coldata.BatchSize(); i++ { diff --git a/pkg/sql/colexec/materializer_test.go b/pkg/sql/colexec/materializer_test.go index 1da4b0afa433..c80aca942725 100644 --- a/pkg/sql/colexec/materializer_test.go +++ b/pkg/sql/colexec/materializer_test.go @@ -117,7 +117,7 @@ func BenchmarkMaterializer(b *testing.B) { if hasNulls { nullProb = nullProbability } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) for _, colVec := range batch.ColVecs() { coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{ Rand: rng, diff --git a/pkg/sql/colexec/mergejoiner.go b/pkg/sql/colexec/mergejoiner.go index e69704e2e8d5..99712f2595a5 100644 --- a/pkg/sql/colexec/mergejoiner.go +++ b/pkg/sql/colexec/mergejoiner.go @@ -540,7 +540,7 @@ func (o *mergeJoinBase) initWithOutputBatchSize(outBatchSize int) { if o.joinType.ShouldIncludeRightColsInOutput() { outputTypes = append(outputTypes, o.right.sourceTypes...) } - o.output = o.unlimitedAllocator.NewMemBatchWithSize(outputTypes, outBatchSize) + o.output = o.unlimitedAllocator.NewMemBatchWithFixedCapacity(outputTypes, outBatchSize) o.left.source.Init() o.right.source.Init() o.outputBatchSize = outBatchSize @@ -554,18 +554,16 @@ func (o *mergeJoinBase) initWithOutputBatchSize(outBatchSize int) { o.unlimitedAllocator, o.left.sourceTypes, o.memoryLimit, o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, ) - o.proberState.lBufferedGroup.firstTuple = make([]coldata.Vec, len(o.left.sourceTypes)) - for colIdx, t := range o.left.sourceTypes { - o.proberState.lBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(t, 1) - } + o.proberState.lBufferedGroup.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( + o.left.sourceTypes, 1, /* capacity */ + ).ColVecs() o.proberState.rBufferedGroup.spillingQueue = newRewindableSpillingQueue( o.unlimitedAllocator, o.right.sourceTypes, o.memoryLimit, o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, ) - o.proberState.rBufferedGroup.firstTuple = make([]coldata.Vec, len(o.right.sourceTypes)) - for colIdx, t := range o.right.sourceTypes { - o.proberState.rBufferedGroup.firstTuple[colIdx] = o.unlimitedAllocator.NewMemColumn(t, 1) - } + o.proberState.rBufferedGroup.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( + o.right.sourceTypes, 1, /* capacity */ + ).ColVecs() o.builderState.lGroups = make([]group, 1) o.builderState.rGroups = make([]group, 1) @@ -596,29 +594,18 @@ func (o *mergeJoinBase) appendToBufferedGroup( } var ( bufferedGroup *mjBufferedGroup - scratchBatch coldata.Batch sourceTypes []*types.T ) if input == &o.left { sourceTypes = o.left.sourceTypes bufferedGroup = &o.proberState.lBufferedGroup - // TODO(yuzefovich): uncomment when spillingQueue actually copies the - // enqueued batches when those are kept in memory. - //if o.scratch.lBufferedGroupBatch == nil { - // o.scratch.lBufferedGroupBatch = o.unlimitedAllocator.NewMemBatch(o.left.sourceTypes) - //} - //scratchBatch = o.scratch.lBufferedGroupBatch } else { sourceTypes = o.right.sourceTypes bufferedGroup = &o.proberState.rBufferedGroup - // TODO(yuzefovich): uncomment when spillingQueue actually copies the - // enqueued batches when those are kept in memory. - //if o.scratch.rBufferedGroupBatch == nil { - // o.scratch.rBufferedGroupBatch = o.unlimitedAllocator.NewMemBatch(o.right.sourceTypes) - //} - //scratchBatch = o.scratch.rBufferedGroupBatch - } - scratchBatch = o.unlimitedAllocator.NewMemBatchWithSize(sourceTypes, groupLength) + } + // TODO(yuzefovich): reuse the same scratch batches when spillingQueue + // actually copies the enqueued batch when those are kept in memory. + scratchBatch := o.unlimitedAllocator.NewMemBatchWithFixedCapacity(sourceTypes, groupLength) if bufferedGroup.numTuples == 0 { o.unlimitedAllocator.PerformOperation(bufferedGroup.firstTuple, func() { for colIdx := range sourceTypes { diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 0634cebdaf95..78de22c9fde0 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -2233,7 +2233,7 @@ func BenchmarkMergeJoiner(b *testing.B) { sourceTypes[colIdx] = types.Int } - batch := testAllocator.NewMemBatch(sourceTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(sourceTypes) queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(b, false /* inMem */) defer cleanup() benchMemAccount := testMemMonitor.MakeBoundAccount() diff --git a/pkg/sql/colexec/offset_test.go b/pkg/sql/colexec/offset_test.go index cf62664729d2..81aed5e4db4f 100644 --- a/pkg/sql/colexec/offset_test.go +++ b/pkg/sql/colexec/offset_test.go @@ -68,7 +68,7 @@ func TestOffset(t *testing.T) { func BenchmarkOffset(b *testing.B) { ctx := context.Background() typs := []*types.T{types.Int, types.Int, types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) source.Init() diff --git a/pkg/sql/colexec/operator.go b/pkg/sql/colexec/operator.go index 61a436072327..990ff35c3b12 100644 --- a/pkg/sql/colexec/operator.go +++ b/pkg/sql/colexec/operator.go @@ -297,7 +297,7 @@ var _ colexecbase.Operator = &singleTupleNoInputOperator{} // batches on all consecutive calls. func NewSingleTupleNoInputOp(allocator *colmem.Allocator) colexecbase.Operator { return &singleTupleNoInputOperator{ - batch: allocator.NewMemBatchWithSize(nil /* types */, 1 /* size */), + batch: allocator.NewMemBatchWithFixedCapacity(nil /* types */, 1 /* size */), } } diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index d58e187ba137..b151651b5533 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -170,12 +170,12 @@ func (a *orderedAggregator) initWithInputAndOutputBatchSize(inputSize, outputSiz // overflow when outputting. a.scratch.inputSize = inputSize * 2 a.scratch.outputSize = outputSize - a.scratch.Batch = a.allocator.NewMemBatchWithSize(a.outputTypes, a.scratch.inputSize) + a.scratch.Batch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, a.scratch.inputSize) for i := 0; i < len(a.outputTypes); i++ { vec := a.scratch.ColVec(i) a.aggregateFuncs[i].Init(a.groupCol, vec) } - a.unsafeBatch = a.allocator.NewMemBatchWithSize(a.outputTypes, outputSize) + a.unsafeBatch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, outputSize) } func (a *orderedAggregator) Init() { diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 9efd189f2440..9f4ff1b63cdd 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -256,7 +256,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { // Init is part of the Operator interface. func (o *OrderedSynchronizer) Init() { o.inputIndices = make([]int, len(o.inputs)) - o.output = o.allocator.NewMemBatch(o.typs) + o.output = o.allocator.NewMemBatchWithMaxCapacity(o.typs) o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) for i, outVec := range o.output.ColVecs() { diff --git a/pkg/sql/colexec/ordered_synchronizer_test.go b/pkg/sql/colexec/ordered_synchronizer_test.go index 195315b006c5..396acaf9cd65 100644 --- a/pkg/sql/colexec/ordered_synchronizer_test.go +++ b/pkg/sql/colexec/ordered_synchronizer_test.go @@ -203,7 +203,7 @@ func BenchmarkOrderedSynchronizer(b *testing.B) { typs := []*types.T{types.Int} batches := make([]coldata.Batch, numInputs) for i := range batches { - batches[i] = testAllocator.NewMemBatch(typs) + batches[i] = testAllocator.NewMemBatchWithMaxCapacity(typs) batches[i].SetLength(coldata.BatchSize()) } for i := int64(0); i < int64(coldata.BatchSize())*numInputs; i++ { diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 0b271a1b37a3..6907e71697ae 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -206,7 +206,7 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { // Init is part of the Operator interface. func (o *OrderedSynchronizer) Init() { o.inputIndices = make([]int, len(o.inputs)) - o.output = o.allocator.NewMemBatch(o.typs) + o.output = o.allocator.NewMemBatchWithMaxCapacity(o.typs) o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) for i, outVec := range o.output.ColVecs() { diff --git a/pkg/sql/colexec/ordinality_test.go b/pkg/sql/colexec/ordinality_test.go index a9d8f1b8ab9e..3f219f9db21c 100644 --- a/pkg/sql/colexec/ordinality_test.go +++ b/pkg/sql/colexec/ordinality_test.go @@ -87,7 +87,7 @@ func BenchmarkOrdinality(b *testing.B) { } typs := []*types.T{types.Int, types.Int, types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) source := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) ordinality, err := createTestOrdinalityOperator(ctx, flowCtx, source, []*types.T{types.Int, types.Int, types.Int}) diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index 3ac524e5c5a2..4a3f31f3193b 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -182,7 +182,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { typs := []*types.T{types.Int} inputs := make([]SynchronizerInput, numInputs) for i := range inputs { - batch := testAllocator.NewMemBatchWithSize(typs, coldata.BatchSize()) + batch := testAllocator.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()) batch.SetLength(coldata.BatchSize()) inputs[i].Op = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) } diff --git a/pkg/sql/colexec/projection_ops_test.go b/pkg/sql/colexec/projection_ops_test.go index c8191864d91c..b53b385f3717 100644 --- a/pkg/sql/colexec/projection_ops_test.go +++ b/pkg/sql/colexec/projection_ops_test.go @@ -198,7 +198,7 @@ func TestRandomComparisons(t *testing.T) { if typ.Family() == types.UuidFamily { bytesFixedLength = 16 } - b := testAllocator.NewMemBatchWithSize(typs, numTuples) + b := testAllocator.NewMemBatchWithFixedCapacity(typs, numTuples) lVec := b.ColVec(0) rVec := b.ColVec(1) ret := b.ColVec(2) @@ -303,7 +303,7 @@ func benchmarkProjOp( ) { ctx := context.Background() rng, _ := randutil.NewPseudoRand() - batch := testAllocator.NewMemBatch(inputTypes) + batch := testAllocator.NewMemBatchWithMaxCapacity(inputTypes) nullProb := 0.0 if hasNulls { nullProb = nullProbability diff --git a/pkg/sql/colexec/relative_rank.eg.go b/pkg/sql/colexec/relative_rank.eg.go index e9ffef6d3322..5e0fafc9ad02 100644 --- a/pkg/sql/colexec/relative_rank.eg.go +++ b/pkg/sql/colexec/relative_rank.eg.go @@ -181,7 +181,7 @@ func (r *percentRankNoPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this // rankIncrement to 1 will update r.rank to 1 on the very first tuple (as @@ -239,7 +239,7 @@ func (r *percentRankNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.scratch = r.allocator.NewMemBatchWithSize(r.inputTypes, n) + r.scratch = r.allocator.NewMemBatchWithFixedCapacity(r.inputTypes, n) r.allocator.PerformOperation(r.scratch.ColVecs(), func() { for colIdx, vec := range r.scratch.ColVecs() { vec.Copy( @@ -381,7 +381,7 @@ func (r *percentRankWithPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this // rankIncrement to 1 will update r.rank to 1 on the very first tuple (as @@ -432,7 +432,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -459,7 +459,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.scratch = r.allocator.NewMemBatchWithSize(r.inputTypes, n) + r.scratch = r.allocator.NewMemBatchWithFixedCapacity(r.inputTypes, n) r.allocator.PerformOperation(r.scratch.ColVecs(), func() { for colIdx, vec := range r.scratch.ColVecs() { vec.Copy( @@ -494,7 +494,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -524,7 +524,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -690,7 +690,7 @@ func (r *cumeDistNoPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) } func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { @@ -736,7 +736,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers @@ -761,7 +761,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.scratch = r.allocator.NewMemBatchWithSize(r.inputTypes, n) + r.scratch = r.allocator.NewMemBatchWithFixedCapacity(r.inputTypes, n) r.allocator.PerformOperation(r.scratch.ColVecs(), func() { for colIdx, vec := range r.scratch.ColVecs() { vec.Copy( @@ -800,7 +800,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -830,7 +830,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -993,7 +993,7 @@ func (r *cumeDistWithPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) } func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { @@ -1039,7 +1039,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -1057,7 +1057,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers @@ -1084,7 +1084,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.scratch = r.allocator.NewMemBatchWithSize(r.inputTypes, n) + r.scratch = r.allocator.NewMemBatchWithFixedCapacity(r.inputTypes, n) r.allocator.PerformOperation(r.scratch.ColVecs(), func() { for colIdx, vec := range r.scratch.ColVecs() { vec.Copy( @@ -1119,7 +1119,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -1149,7 +1149,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -1187,7 +1187,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -1217,7 +1217,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { diff --git a/pkg/sql/colexec/relative_rank_tmpl.go b/pkg/sql/colexec/relative_rank_tmpl.go index bacb43cc8e3f..84bd57b293c9 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -144,7 +144,7 @@ func _COMPUTE_PARTITIONS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -182,7 +182,7 @@ func _COMPUTE_PEER_GROUPS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -300,7 +300,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, ) - r.output = r.allocator.NewMemBatch(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // {{if .IsPercentRank}} // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this @@ -358,7 +358,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -378,7 +378,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatch([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers @@ -411,7 +411,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.scratch = r.allocator.NewMemBatchWithSize(r.inputTypes, n) + r.scratch = r.allocator.NewMemBatchWithFixedCapacity(r.inputTypes, n) r.allocator.PerformOperation(r.scratch.ColVecs(), func() { for colIdx, vec := range r.scratch.ColVecs() { vec.Copy( diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index ed69257710d8..3c069cef14a0 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -419,7 +419,7 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele for toAppend := len(selection); toAppend > 0; { if o.mu.pendingBatch == nil { - o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithSize(o.types, o.testingKnobs.outputBatchSize) + o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithFixedCapacity(o.types, o.testingKnobs.outputBatchSize) } available := o.testingKnobs.outputBatchSize - o.mu.pendingBatch.Length() numAppended := toAppend diff --git a/pkg/sql/colexec/routers_test.go b/pkg/sql/colexec/routers_test.go index 0bd6e691d988..6ffed32e5ca1 100644 --- a/pkg/sql/colexec/routers_test.go +++ b/pkg/sql/colexec/routers_test.go @@ -705,7 +705,7 @@ func TestHashRouterCancellation(t *testing.T) { typs := []*types.T{types.Int} // Never-ending input of 0s. - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) in := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) @@ -1204,7 +1204,7 @@ func BenchmarkHashRouter(b *testing.B) { // Use only one type. Note: the more types you use, the more you inflate the // numbers. typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) diff --git a/pkg/sql/colexec/select_in_test.go b/pkg/sql/colexec/select_in_test.go index 39490f796e0f..cb3b2eb6aad5 100644 --- a/pkg/sql/colexec/select_in_test.go +++ b/pkg/sql/colexec/select_in_test.go @@ -100,7 +100,7 @@ func TestSelectInInt64(t *testing.T) { func benchmarkSelectInInt64(b *testing.B, useSelectionVector bool, hasNulls bool) { ctx := context.Background() typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) col1 := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { diff --git a/pkg/sql/colexec/selection_ops_test.go b/pkg/sql/colexec/selection_ops_test.go index f9810e1d78ac..94876e6432a9 100644 --- a/pkg/sql/colexec/selection_ops_test.go +++ b/pkg/sql/colexec/selection_ops_test.go @@ -158,7 +158,7 @@ func benchmarkSelLTInt64Int64ConstOp(b *testing.B, useSelectionVector bool, hasN ctx := context.Background() typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) col := batch.ColVec(0).Int64() for i := 0; i < coldata.BatchSize(); i++ { if float64(i) < float64(coldata.BatchSize())*selectivity { @@ -215,7 +215,7 @@ func benchmarkSelLTInt64Int64Op(b *testing.B, useSelectionVector bool, hasNulls ctx := context.Background() typs := []*types.T{types.Int, types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) col1 := batch.ColVec(0).Int64() col2 := batch.ColVec(1).Int64() for i := 0; i < coldata.BatchSize(); i++ { diff --git a/pkg/sql/colexec/simple_project_test.go b/pkg/sql/colexec/simple_project_test.go index 766e899cf0b2..ef33bf2bf943 100644 --- a/pkg/sql/colexec/simple_project_test.go +++ b/pkg/sql/colexec/simple_project_test.go @@ -78,7 +78,7 @@ func TestSimpleProjectOp(t *testing.T) { t.Run("RedundantProjectionIsNotPlanned", func(t *testing.T) { typs := []*types.T{types.Int, types.Int} - input := newFiniteBatchSource(testAllocator.NewMemBatch(typs), typs, 1 /* usableCount */) + input := newFiniteBatchSource(testAllocator.NewMemBatchWithMaxCapacity(typs), typs, 1 /* usableCount */) projectOp := NewSimpleProjectOp(input, len(typs), []uint32{0, 1}) require.IsType(t, input, projectOp) }) diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 3e38f3ae8850..0040946bd143 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -127,10 +127,8 @@ func newAllSpooler( func (p *allSpooler) init() { p.input.Init() - p.bufferedTuples = newAppendOnlyBufferedBatch( - p.allocator, p.inputTypes, 0, /* initialSize */ - ) - p.windowedBatch = p.allocator.NewMemBatchWithSize(p.inputTypes, 0 /* size */) + p.bufferedTuples = newAppendOnlyBufferedBatch(p.allocator, p.inputTypes) + p.windowedBatch = p.allocator.NewMemBatchWithFixedCapacity(p.inputTypes, 0 /* size */) } func (p *allSpooler) spool(ctx context.Context) { @@ -398,7 +396,7 @@ func (p *sortOp) sort(ctx context.Context) { func (p *sortOp) resetOutput() { if p.output == nil { - p.output = p.allocator.NewMemBatch(p.inputTypes) + p.output = p.allocator.NewMemBatchWithMaxCapacity(p.inputTypes) } else { p.output.ResetInternalBatch() } diff --git a/pkg/sql/colexec/sort_chunks.go b/pkg/sql/colexec/sort_chunks.go index 2a7607764350..8d563a06e140 100644 --- a/pkg/sql/colexec/sort_chunks.go +++ b/pkg/sql/colexec/sort_chunks.go @@ -278,9 +278,7 @@ func newChunker( func (s *chunker) init() { s.input.Init() - s.bufferedTuples = newAppendOnlyBufferedBatch( - s.allocator, s.inputTypes, 0, /* initialSize */ - ) + s.bufferedTuples = newAppendOnlyBufferedBatch(s.allocator, s.inputTypes) s.partitionCol = make([]bool, coldata.BatchSize()) s.chunks = make([]int, 0, 16) } diff --git a/pkg/sql/colexec/sort_chunks_test.go b/pkg/sql/colexec/sort_chunks_test.go index 7cee0eafcaeb..c8bceb096b2a 100644 --- a/pkg/sql/colexec/sort_chunks_test.go +++ b/pkg/sql/colexec/sort_chunks_test.go @@ -266,7 +266,7 @@ func BenchmarkSortChunks(b *testing.B) { for i := range typs { typs[i] = types.Int } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) ordCols := make([]execinfrapb.Ordering_Column, nCols) for i := range ordCols { diff --git a/pkg/sql/colexec/sort_test.go b/pkg/sql/colexec/sort_test.go index 6b59cd791a48..af5f3444ca9c 100644 --- a/pkg/sql/colexec/sort_test.go +++ b/pkg/sql/colexec/sort_test.go @@ -294,7 +294,7 @@ func BenchmarkSort(b *testing.B) { for i := range typs { typs[i] = types.Int } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) ordCols := make([]execinfrapb.Ordering_Column, nCols) for i := range ordCols { @@ -343,7 +343,7 @@ func BenchmarkAllSpooler(b *testing.B) { for i := range typs { typs[i] = types.Int } - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) for i := 0; i < nCols; i++ { col := batch.ColVec(i).Int64() diff --git a/pkg/sql/colexec/sorttopk.go b/pkg/sql/colexec/sorttopk.go index 99593c59a17b..2962dcb84d18 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -95,9 +95,7 @@ type topKSorter struct { func (t *topKSorter) Init() { t.input.Init() - t.topK = newAppendOnlyBufferedBatch( - t.allocator, t.inputTypes, 0, /* initialSize */ - ) + t.topK = newAppendOnlyBufferedBatch(t.allocator, t.inputTypes) t.comparators = make([]vecComparator, len(t.inputTypes)) for i, typ := range t.inputTypes { t.comparators[i] = GetVecComparator(typ, 2) @@ -199,7 +197,7 @@ func (t *topKSorter) spool(ctx context.Context) { func (t *topKSorter) resetOutput() { if t.output == nil { - t.output = t.allocator.NewMemBatchWithSize(t.inputTypes, coldata.BatchSize()) + t.output = t.allocator.NewMemBatchWithMaxCapacity(t.inputTypes) } else { t.output.ResetInternalBatch() } diff --git a/pkg/sql/colexec/spilling_queue.go b/pkg/sql/colexec/spilling_queue.go index 01a24dd394ff..7136e76501b2 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -99,7 +99,7 @@ func newSpillingQueue( items: make([]coldata.Batch, itemsLen), diskQueueCfg: cfg, fdSemaphore: fdSemaphore, - dequeueScratch: unlimitedAllocator.NewMemBatchWithSize(typs, coldata.BatchSize()), + dequeueScratch: unlimitedAllocator.NewMemBatchWithMaxCapacity(typs), diskAcc: diskAcc, } } diff --git a/pkg/sql/colexec/stats_test.go b/pkg/sql/colexec/stats_test.go index b1e1f303aae8..73698de88db3 100644 --- a/pkg/sql/colexec/stats_test.go +++ b/pkg/sql/colexec/stats_test.go @@ -150,7 +150,7 @@ func TestVectorizedStatsCollector(t *testing.T) { func makeFiniteChunksSourceWithBatchSize(nBatches int, batchSize int) colexecbase.Operator { typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatchWithSize(typs, batchSize) + batch := testAllocator.NewMemBatchWithFixedCapacity(typs, batchSize) vec := batch.ColVec(0).Int64() for i := 0; i < batchSize; i++ { vec[i] = int64(i) diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index 88fd491ffbf6..9e0520444451 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -165,7 +165,7 @@ func (a *arrowTestOperator) Next(ctx context.Context) coldata.Batch { if err := a.r.Deserialize(&arrowDataOut, buf.Bytes()); err != nil { colexecerror.InternalError(err) } - batchOut := testAllocator.NewMemBatchWithSize(a.typs, coldata.BatchSize()) + batchOut := testAllocator.NewMemBatchWithFixedCapacity(a.typs, coldata.BatchSize()) if err := a.c.ArrowToBatch(arrowDataOut, batchOut); err != nil { colexecerror.InternalError(err) } diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index 64fbc3add178..d7059c8ce1ea 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -44,7 +44,7 @@ func NewUnorderedDistinct( OneInputNode: NewOneInputNode(input), allocator: allocator, ht: ht, - output: allocator.NewMemBatch(typs), + output: allocator.NewMemBatchWithMaxCapacity(typs), } } diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index 431a61666bcd..6cdd605bd904 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -91,7 +91,7 @@ func newPartitionerToOperator( return &partitionerToOperator{ partitioner: partitioner, partitionIdx: partitionIdx, - batch: allocator.NewMemBatch(types), + batch: allocator.NewMemBatchWithMaxCapacity(types), } } @@ -119,9 +119,9 @@ func (p *partitionerToOperator) Next(ctx context.Context) coldata.Batch { } func newAppendOnlyBufferedBatch( - allocator *colmem.Allocator, typs []*types.T, initialSize int, + allocator *colmem.Allocator, typs []*types.T, ) *appendOnlyBufferedBatch { - batch := allocator.NewMemBatchWithSize(typs, initialSize) + batch := allocator.NewMemBatchWithFixedCapacity(typs, 0 /* capacity */) return &appendOnlyBufferedBatch{ Batch: batch, colVecs: batch.ColVecs(), diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index cc432b5afeb7..2980cb63bad4 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -722,7 +722,7 @@ func (s *opTestInput) Init() { } s.typs = extrapolateTypesFromTuples(s.tuples) } - s.batch = testAllocator.NewMemBatch(s.typs) + s.batch = testAllocator.NewMemBatchWithMaxCapacity(s.typs) s.selection = make([]int, coldata.BatchSize()) for i := range s.selection { @@ -898,7 +898,7 @@ func (s *opFixedSelTestInput) Init() { s.typs = extrapolateTypesFromTuples(s.tuples) } - s.batch = testAllocator.NewMemBatch(s.typs) + s.batch = testAllocator.NewMemBatchWithMaxCapacity(s.typs) tupleLen := len(s.tuples[0]) for _, i := range s.sel { if len(s.tuples[i]) != tupleLen { @@ -1317,7 +1317,7 @@ func TestRepeatableBatchSource(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batchLen := 10 if coldata.BatchSize() < batchLen { batchLen = coldata.BatchSize() @@ -1342,7 +1342,7 @@ func TestRepeatableBatchSourceWithFixedSel(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) rng, _ := randutil.NewPseudoRand() batchSize := 10 if batchSize > coldata.BatchSize() { @@ -1417,7 +1417,7 @@ func newChunkingBatchSource(typs []*types.T, cols []coldata.Vec, len int) *chunk } func (c *chunkingBatchSource) Init() { - c.batch = testAllocator.NewMemBatch(c.typs) + c.batch = testAllocator.NewMemBatchWithMaxCapacity(c.typs) for i := range c.cols { c.batch.ColVec(i).SetCol(c.cols[i].Col()) c.batch.ColVec(i).SetNulls(c.cols[i].Nulls()) diff --git a/pkg/sql/colexecbase/testutils.go b/pkg/sql/colexecbase/testutils.go index 9f3af1fa1a2d..bd55cddc2f30 100644 --- a/pkg/sql/colexecbase/testutils.go +++ b/pkg/sql/colexecbase/testutils.go @@ -91,7 +91,7 @@ func NewRepeatableBatchSource( } numToCopy = maxIdx + 1 } - output := allocator.NewMemBatchWithSize(typs, numToCopy) + output := allocator.NewMemBatchWithFixedCapacity(typs, numToCopy) src := &RepeatableBatchSource{ colVecs: batch.ColVecs(), typs: typs, diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 17d6c46d6316..93688bff8d34 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -328,7 +328,7 @@ func (rf *cFetcher) Init( } sort.Ints(table.neededColsList) - rf.machine.batch = allocator.NewMemBatch(typs) + rf.machine.batch = allocator.NewMemBatchWithMaxCapacity(typs) rf.machine.colvecs = rf.machine.batch.ColVecs() // If the fetcher is requested to produce a timestamp column, pull out the // column as a decimal and save it. diff --git a/pkg/sql/colflow/colrpc/colrpc_test.go b/pkg/sql/colflow/colrpc/colrpc_test.go index d924f85a92e1..7ed97fcfd257 100644 --- a/pkg/sql/colflow/colrpc/colrpc_test.go +++ b/pkg/sql/colflow/colrpc/colrpc_test.go @@ -286,7 +286,7 @@ func TestOutboxInbox(t *testing.T) { if outputBatch == coldata.ZeroBatch { outputBatches.Add(coldata.ZeroBatch, typs) } else { - batchCopy := testAllocator.NewMemBatchWithSize(typs, outputBatch.Length()) + batchCopy := testAllocator.NewMemBatchWithFixedCapacity(typs, outputBatch.Length()) testAllocator.PerformOperation(batchCopy.ColVecs(), func() { for i := range typs { batchCopy.ColVec(i).Append( @@ -560,7 +560,7 @@ func BenchmarkOutboxInbox(b *testing.B) { typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatch(typs) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) input := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) @@ -621,7 +621,7 @@ func TestOutboxStreamIDPropagation(t *testing.T) { nextDone := make(chan struct{}) input := &colexecbase.CallbackOperator{NextCb: func(ctx context.Context) coldata.Batch { - b := testAllocator.NewMemBatchWithSize(typs, 0) + b := testAllocator.NewMemBatchWithFixedCapacity(typs, 0) b.SetLength(0) inTags = logtags.FromContext(ctx) nextDone <- struct{}{} diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 0cd45f7e9975..7e8686644eb2 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -130,7 +130,7 @@ func NewInbox( flowCtx: ctx, } i.scratch.data = make([]*array.Data, len(typs)) - i.scratch.b = allocator.NewMemBatch(typs) + i.scratch.b = allocator.NewMemBatchWithMaxCapacity(typs) return i, nil } diff --git a/pkg/sql/colflow/colrpc/outbox_test.go b/pkg/sql/colflow/colrpc/outbox_test.go index 42b92abd9d6d..405d9f1ccfad 100644 --- a/pkg/sql/colflow/colrpc/outbox_test.go +++ b/pkg/sql/colflow/colrpc/outbox_test.go @@ -114,7 +114,7 @@ func TestOutboxDrainsMetadataSources(t *testing.T) { ) require.NoError(t, err) - b := testAllocator.NewMemBatch(typs) + b := testAllocator.NewMemBatchWithMaxCapacity(typs) b.SetLength(0) input.Add(b, typs) diff --git a/pkg/sql/colflow/vectorized_flow_shutdown_test.go b/pkg/sql/colflow/vectorized_flow_shutdown_test.go index e167e1e15169..9e86c8519783 100644 --- a/pkg/sql/colflow/vectorized_flow_shutdown_test.go +++ b/pkg/sql/colflow/vectorized_flow_shutdown_test.go @@ -297,7 +297,7 @@ func TestVectorizedFlowShutdown(t *testing.T) { sourceMemAccount := testMemMonitor.MakeBoundAccount() defer sourceMemAccount.Close(ctxRemote) remoteAllocator := colmem.NewAllocator(ctxRemote, &sourceMemAccount, testColumnFactory) - batch := remoteAllocator.NewMemBatch(typs) + batch := remoteAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) runOutboxInbox( ctxRemote, diff --git a/pkg/sql/colflow/vectorized_flow_space_test.go b/pkg/sql/colflow/vectorized_flow_space_test.go index f7d1b50fd921..a9f8d78e6206 100644 --- a/pkg/sql/colflow/vectorized_flow_space_test.go +++ b/pkg/sql/colflow/vectorized_flow_space_test.go @@ -196,7 +196,7 @@ func TestVectorizeAllocatorSpaceError(t *testing.T) { } typs := []*types.T{types.Int} - batch := testAllocator.NewMemBatchWithSize(typs, 1 /* size */) + batch := testAllocator.NewMemBatchWithFixedCapacity(typs, 1 /* size */) for _, tc := range testCases { for _, success := range []bool{true, false} { expectNoMemoryError := success || tc.spillingSupported diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index 5ce0a815f653..93c813e1ecdc 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -89,30 +89,31 @@ func NewAllocator( } } -// NewMemBatch allocates a new in-memory coldata.Batch. -func (a *Allocator) NewMemBatch(typs []*types.T) coldata.Batch { - return a.NewMemBatchWithSize(typs, coldata.BatchSize()) +// NewMemBatchWithMaxCapacity allocates a new in-memory coldata.Batch of +// coldata.BatchSize() capacity. +func (a *Allocator) NewMemBatchWithMaxCapacity(typs []*types.T) coldata.Batch { + return a.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()) } -// NewMemBatchWithSize allocates a new in-memory coldata.Batch with the given -// column size. -func (a *Allocator) NewMemBatchWithSize(typs []*types.T, size int) coldata.Batch { - estimatedMemoryUsage := selVectorSize(size) + int64(EstimateBatchSizeBytes(typs, size)) +// NewMemBatchWithFixedCapacity allocates a new in-memory coldata.Batch with +// the given capacity. +func (a *Allocator) NewMemBatchWithFixedCapacity(typs []*types.T, capacity int) coldata.Batch { + estimatedMemoryUsage := selVectorSize(capacity) + int64(EstimateBatchSizeBytes(typs, capacity)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - return coldata.NewMemBatchWithSize(typs, size, a.factory) + return coldata.NewMemBatchWithCapacity(typs, capacity, a.factory) } // NewMemBatchNoCols creates a "skeleton" of new in-memory coldata.Batch. It // allocates memory for the selection vector but does *not* allocate any memory // for the column vectors - those will have to be added separately. -func (a *Allocator) NewMemBatchNoCols(types []*types.T, size int) coldata.Batch { - estimatedMemoryUsage := selVectorSize(size) +func (a *Allocator) NewMemBatchNoCols(types []*types.T, capacity int) coldata.Batch { + estimatedMemoryUsage := selVectorSize(capacity) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - return coldata.NewMemBatchNoCols(types, size) + return coldata.NewMemBatchNoCols(types, capacity) } // RetainBatch adds the size of the batch to the memory account. This shouldn't @@ -160,13 +161,15 @@ func (a *Allocator) ReleaseBatch(b coldata.Batch) { b.SetSelection(usesSel) } -// NewMemColumn returns a new coldata.Vec, initialized with a length. -func (a *Allocator) NewMemColumn(t *types.T, n int) coldata.Vec { - estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, n)) +// NewMemColumn returns a new coldata.Vec of the desired capacity. +// NOTE: consider whether you should be using MaybeAppendColumn or +// NewMemBatchWith* methods or colexec.DynamicBatchSizeHelper struct. +func (a *Allocator) NewMemColumn(t *types.T, capacity int) coldata.Vec { + estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, capacity)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - return coldata.NewMemColumn(t, n, a.factory) + return coldata.NewMemColumn(t, capacity, a.factory) } // MaybeAppendColumn might append a newly allocated coldata.Vec of the given @@ -216,11 +219,11 @@ func (a *Allocator) MaybeAppendColumn(b coldata.Batch, t *types.T, colIdx int) { t, colIdx, width, )) } - estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, coldata.BatchSize())) + estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, b.Capacity())) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - b.AppendCol(a.NewMemColumn(t, coldata.BatchSize())) + b.AppendCol(a.NewMemColumn(t, b.Capacity())) } // PerformOperation executes 'operation' (that somehow modifies 'destVecs') and diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index d3b7d26cad68..981a03013ad5 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1561,11 +1561,11 @@ func TestLint(t *testing.T) { "-nE", // We prohibit usage of: // - coldata.NewMemBatch - // - coldata.NewMemBatchWithSize + // - coldata.NewMemBatchWithCapacity // - coldata.NewMemColumn // - coldata.Batch.AppendCol // TODO(yuzefovich): prohibit call to coldata.NewMemBatchNoCols. - fmt.Sprintf(`(coldata\.NewMem(Batch|BatchWithSize|Column)|\.AppendCol)\(`), + fmt.Sprintf(`(coldata\.NewMem(Batch|BatchWithCapacity|Column)|\.AppendCol)\(`), "--", // TODO(yuzefovich): prohibit calling coldata.* methods from other // sql/col* packages. diff --git a/pkg/workload/csv.go b/pkg/workload/csv.go index 0764cb7d9bcd..5afe69293e27 100644 --- a/pkg/workload/csv.go +++ b/pkg/workload/csv.go @@ -39,7 +39,7 @@ const ( func WriteCSVRows( ctx context.Context, w io.Writer, table Table, rowStart, rowEnd int, sizeBytesLimit int64, ) (rowBatchIdx int, err error) { - cb := coldata.NewMemBatchWithSize(nil /* types */, 0 /* size */, coldata.StandardColumnFactory) + cb := coldata.NewMemBatchWithCapacity(nil /* typs */, 0 /* capacity */, coldata.StandardColumnFactory) var a bufalloc.ByteAllocator bytesWrittenW := &bytesWrittenWriter{w: w} @@ -91,7 +91,7 @@ type csvRowsReader struct { func (r *csvRowsReader) Read(p []byte) (n int, err error) { if r.cb == nil { - r.cb = coldata.NewMemBatchWithSize(nil /* types */, 0 /* size */, coldata.StandardColumnFactory) + r.cb = coldata.NewMemBatchWithCapacity(nil /* typs */, 0 /* capacity */, coldata.StandardColumnFactory) } for { diff --git a/pkg/workload/workload.go b/pkg/workload/workload.go index dc4d08b89585..12dba8317157 100644 --- a/pkg/workload/workload.go +++ b/pkg/workload/workload.go @@ -252,7 +252,7 @@ func TypedTuples(count int, typs []*types.T, fn func(int) []interface{}) Batched // heavy. In performance-critical code, FillBatch should be used directly, // instead. func (b BatchedTuples) BatchRows(batchIdx int) [][]interface{} { - cb := coldata.NewMemBatchWithSize(nil, 0, coldata.StandardColumnFactory) + cb := coldata.NewMemBatchWithCapacity(nil, 0, coldata.StandardColumnFactory) var a bufalloc.ByteAllocator b.FillBatch(batchIdx, cb, &a) return ColBatchToRows(cb) From 2bf2f9790a446fce564ec6fc2b4ffdbd2e4185a9 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 5 Aug 2020 19:32:32 -0700 Subject: [PATCH 2/3] colexec: remove custom input/output batch size logic from few places Ordered aggregator, hash and merge joiners, and hash router had custom input/output batch size logic that was put in place in order to increase testing. This, however, is no longer required since we now randomize `coldata.BatchSize()` value during the test runs, so that custom logic is now removed. Additionally, this commit removes several unit tests of the merge joiner which are now exact copies of each other (previously, they had different output batch size set). One notable change is that this commit removes a tiny optimization from the merge joiner when there are no output columns (meaning we have a COUNT query). This work has been done in order to ease follow-up work on the dynamic batch sizes. Release note: None --- pkg/sql/colexec/aggregators_test.go | 66 +-- pkg/sql/colexec/hashjoiner.eg.go | 8 +- pkg/sql/colexec/hashjoiner.go | 12 +- pkg/sql/colexec/hashjoiner_test.go | 47 +- pkg/sql/colexec/hashjoiner_tmpl.go | 4 +- pkg/sql/colexec/mergejoiner.go | 21 +- pkg/sql/colexec/mergejoiner_exceptall.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_fullouter.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_inner.eg.go | 375 +++++++------ .../colexec/mergejoiner_intersectall.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_leftanti.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_leftouter.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_leftsemi.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_rightouter.eg.go | 375 +++++++------ pkg/sql/colexec/mergejoiner_test.go | 519 ++++++------------ pkg/sql/colexec/mergejoiner_tmpl.go | 27 +- pkg/sql/colexec/ordered_aggregator.go | 40 +- .../parallel_unordered_synchronizer_test.go | 2 +- pkg/sql/colexec/relative_rank.eg.go | 16 +- pkg/sql/colexec/relative_rank_tmpl.go | 6 +- pkg/sql/colexec/routers.go | 13 +- pkg/sql/colexec/routers_test.go | 10 +- pkg/sql/colexec/spilling_queue.go | 6 +- pkg/sql/colexec/spilling_queue_test.go | 6 +- pkg/sql/colexec/types_integration_test.go | 2 +- pkg/sql/colexec/utils_test.go | 54 +- 26 files changed, 1787 insertions(+), 2072 deletions(-) diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 64936aeb3cfd..54a1eddb4e53 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -54,11 +54,8 @@ type aggregatorTestCase struct { input tuples unorderedInput bool expected tuples - // {output}BatchSize() if not 0 are passed in to NewOrderedAggregator to - // divide input/output batches. - batchSize int - outputBatchSize int - name string + inputBatchSize int + name string // convToDecimal will convert any float64s to apd.Decimals. If a string is // encountered, a best effort is made to convert that string to an @@ -147,11 +144,8 @@ func (tc *aggregatorTestCase) init() error { if tc.typs == nil { tc.typs = defaultTyps } - if tc.batchSize == 0 { - tc.batchSize = coldata.BatchSize() - } - if tc.outputBatchSize == 0 { - tc.outputBatchSize = coldata.BatchSize() + if tc.inputBatchSize == 0 { + tc.inputBatchSize = coldata.BatchSize() } aggregations := make([]execinfrapb.AggregatorSpec_Aggregation, len(tc.aggFns)) for i, aggFn := range tc.aggFns { @@ -179,8 +173,7 @@ func TestAggregatorOneFunc(t *testing.T) { expected: tuples{ {1}, }, - name: "OneTuple", - outputBatchSize: 4, + name: "OneTuple", }, { input: tuples{ @@ -205,8 +198,8 @@ func TestAggregatorOneFunc(t *testing.T) { {4}, {5}, }, - batchSize: 2, - name: "MultiGroup", + inputBatchSize: 2, + name: "MultiGroup", }, { input: tuples{ @@ -220,8 +213,8 @@ func TestAggregatorOneFunc(t *testing.T) { {6}, {9}, }, - batchSize: 1, - name: "CarryBetweenInputBatches", + inputBatchSize: 1, + name: "CarryBetweenInputBatches", }, { input: tuples{ @@ -237,9 +230,8 @@ func TestAggregatorOneFunc(t *testing.T) { {5}, {6}, }, - batchSize: 2, - outputBatchSize: 1, - name: "CarryBetweenOutputBatches", + inputBatchSize: 2, + name: "CarryBetweenOutputBatches", }, { input: tuples{ @@ -265,9 +257,8 @@ func TestAggregatorOneFunc(t *testing.T) { {7}, {8}, }, - batchSize: 3, - outputBatchSize: 1, - name: "CarryBetweenInputAndOutputBatches", + inputBatchSize: 3, + name: "CarryBetweenInputAndOutputBatches", }, { input: tuples{ @@ -279,10 +270,9 @@ func TestAggregatorOneFunc(t *testing.T) { expected: tuples{ {10}, }, - batchSize: 1, - outputBatchSize: 1, - name: "NoGroupingCols", - groupCols: []uint32{}, + inputBatchSize: 1, + name: "NoGroupingCols", + groupCols: []uint32{}, }, { input: tuples{ @@ -294,12 +284,11 @@ func TestAggregatorOneFunc(t *testing.T) { expected: tuples{ {10}, }, - batchSize: 1, - outputBatchSize: 1, - name: "UnusedInputColumns", - typs: []*types.T{types.Int, types.Int, types.Int}, - groupCols: []uint32{1, 2}, - aggCols: [][]uint32{{0}}, + inputBatchSize: 1, + name: "UnusedInputColumns", + typs: []*types.T{types.Int, types.Int, types.Int}, + groupCols: []uint32{1, 2}, + aggCols: [][]uint32{{0}}, }, { input: tuples{ @@ -362,7 +351,7 @@ func TestAggregatorOneFunc(t *testing.T) { require.NoError(t, err) if !tc.unorderedInput { log.Infof(ctx, "%s", tc.name) - tupleSource := newOpTestInput(tc.batchSize, tc.input, tc.typs) + tupleSource := newOpTestInput(tc.inputBatchSize, tc.input, tc.typs) a, err := NewOrderedAggregator( testAllocator, tupleSource, tc.typs, tc.spec, &evalCtx, constructors, constArguments, outputTypes, false, /* isScalar */ @@ -372,9 +361,6 @@ func TestAggregatorOneFunc(t *testing.T) { } out := newOpTestOutput(a, tc.expected) - // Explicitly reinitialize the aggregator with the given output batch - // size. - a.(*orderedAggregator).initWithInputAndOutputBatchSize(tc.batchSize, tc.outputBatchSize) if err := out.VerifyAnyOrder(); err != nil { t.Fatal(err) } @@ -518,10 +504,10 @@ func TestAggregatorMultiFunc(t *testing.T) { {2, 1.0, "1.0", 2.0, 6.0}, {2, 2.0, "2.0", 6.0, 6.0}, }, - batchSize: 1, - typs: []*types.T{types.Int, types.Decimal, types.Bytes, types.Decimal}, - name: "MultiGroupColsWithPointerTypes", - groupCols: []uint32{0, 1, 2}, + inputBatchSize: 1, + typs: []*types.T{types.Int, types.Decimal, types.Bytes, types.Decimal}, + name: "MultiGroupColsWithPointerTypes", + groupCols: []uint32{0, 1, 2}, aggCols: [][]uint32{ {0}, {1}, {2}, {3}, {3}, }, diff --git a/pkg/sql/colexec/hashjoiner.eg.go b/pkg/sql/colexec/hashjoiner.eg.go index b4f05d22fb03..7c92c3fcf223 100644 --- a/pkg/sql/colexec/hashjoiner.eg.go +++ b/pkg/sql/colexec/hashjoiner.eg.go @@ -24,7 +24,7 @@ func collectProbeOuter_false( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -70,7 +70,7 @@ func collectProbeOuter_true( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -116,7 +116,7 @@ func collectProbeNoOuter_false( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -148,7 +148,7 @@ func collectProbeNoOuter_true( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index f8578b82b4fa..7418a374309d 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -173,9 +173,6 @@ type hashJoiner struct { // output stores the resulting output batch that is constructed and returned // for every input batch during the probe phase. output coldata.Batch - // outputBatchSize specifies the desired length of the output batch which by - // default is coldata.BatchSize() but can be varied in tests. - outputBatchSize int // probeState is used in hjProbing state. probeState struct { @@ -324,7 +321,7 @@ func (hj *hashJoiner) emitUnmatched() { nResults := 0 - for nResults < hj.outputBatchSize && hj.emittingUnmatchedState.rowIdx < hj.ht.vals.Length() { + for nResults < hj.output.Capacity() && hj.emittingUnmatchedState.rowIdx < hj.ht.vals.Length() { if !hj.probeState.buildRowMatched[hj.emittingUnmatchedState.rowIdx] { hj.probeState.buildIdx[nResults] = hj.emittingUnmatchedState.rowIdx nResults++ @@ -688,10 +685,9 @@ func NewHashJoiner( allocator *colmem.Allocator, spec HashJoinerSpec, leftSource, rightSource colexecbase.Operator, ) colexecbase.Operator { hj := &hashJoiner{ - twoInputNode: newTwoInputNode(leftSource, rightSource), - allocator: allocator, - spec: spec, - outputBatchSize: coldata.BatchSize(), + twoInputNode: newTwoInputNode(leftSource, rightSource), + allocator: allocator, + spec: spec, } hj.probeState.buildIdx = make([]int, coldata.BatchSize()) hj.probeState.probeIdx = make([]int, coldata.BatchSize()) diff --git a/pkg/sql/colexec/hashjoiner_test.go b/pkg/sql/colexec/hashjoiner_test.go index 2010543031f2..096be9673298 100644 --- a/pkg/sql/colexec/hashjoiner_test.go +++ b/pkg/sql/colexec/hashjoiner_test.go @@ -992,35 +992,24 @@ func TestHashJoiner(t *testing.T) { Cfg: &execinfra.ServerConfig{Settings: st}, } - for _, outputBatchSize := range []int{1, 17, coldata.BatchSize()} { - if outputBatchSize > coldata.BatchSize() { - // It is possible for varied coldata.BatchSize() to be smaller than - // requested outputBatchSize. Such configuration is invalid, and we skip - // it. - continue - } - for _, tcs := range [][]*joinTestCase{hjTestCases, mjTestCases} { - for _, tc := range tcs { - for _, tc := range tc.mutateTypes() { - runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { - spec := createSpecForHashJoiner(tc) - args := &NewColOperatorArgs{ - Spec: spec, - Inputs: sources, - StreamingMemAccount: testMemAcc, - } - args.TestingKnobs.UseStreamingMemAccountForBuffering = true - args.TestingKnobs.DiskSpillingDisabled = true - result, err := TestNewColOperator(ctx, flowCtx, args) - if err != nil { - return nil, err - } - if hj, ok := result.Op.(*hashJoiner); ok { - hj.outputBatchSize = outputBatchSize - } - return result.Op, nil - }) - } + for _, tcs := range [][]*joinTestCase{hjTestCases, mjTestCases} { + for _, tc := range tcs { + for _, tc := range tc.mutateTypes() { + runHashJoinTestCase(t, tc, func(sources []colexecbase.Operator) (colexecbase.Operator, error) { + spec := createSpecForHashJoiner(tc) + args := &NewColOperatorArgs{ + Spec: spec, + Inputs: sources, + StreamingMemAccount: testMemAcc, + } + args.TestingKnobs.UseStreamingMemAccountForBuffering = true + args.TestingKnobs.DiskSpillingDisabled = true + result, err := TestNewColOperator(ctx, flowCtx, args) + if err != nil { + return nil, err + } + return result.Op, nil + }) } } } diff --git a/pkg/sql/colexec/hashjoiner_tmpl.go b/pkg/sql/colexec/hashjoiner_tmpl.go index 1df7a73fc06f..00a96abee4d2 100644 --- a/pkg/sql/colexec/hashjoiner_tmpl.go +++ b/pkg/sql/colexec/hashjoiner_tmpl.go @@ -30,7 +30,7 @@ func collectProbeOuter( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -71,7 +71,7 @@ func collectProbeNoOuter( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.outputBatchSize { + if nResults >= hj.output.Capacity() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults diff --git a/pkg/sql/colexec/mergejoiner.go b/pkg/sql/colexec/mergejoiner.go index 99712f2595a5..57c26c8abad5 100644 --- a/pkg/sql/colexec/mergejoiner.go +++ b/pkg/sql/colexec/mergejoiner.go @@ -12,7 +12,6 @@ package colexec import ( "context" - "math" "unsafe" "github.com/cockroachdb/cockroach/pkg/col/coldata" @@ -489,8 +488,7 @@ type mergeJoinBase struct { right mergeJoinInput // Output buffer definition. - output coldata.Batch - outputBatchSize int + output coldata.Batch // outputReady is a flag to indicate that merge joiner is ready to emit an // output batch. outputReady bool @@ -532,34 +530,23 @@ func (o *mergeJoinBase) InternalMemoryUsage() int { } func (o *mergeJoinBase) Init() { - o.initWithOutputBatchSize(coldata.BatchSize()) -} - -func (o *mergeJoinBase) initWithOutputBatchSize(outBatchSize int) { outputTypes := append([]*types.T{}, o.left.sourceTypes...) if o.joinType.ShouldIncludeRightColsInOutput() { outputTypes = append(outputTypes, o.right.sourceTypes...) } - o.output = o.unlimitedAllocator.NewMemBatchWithFixedCapacity(outputTypes, outBatchSize) + o.output = o.unlimitedAllocator.NewMemBatchWithMaxCapacity(outputTypes) o.left.source.Init() o.right.source.Init() - o.outputBatchSize = outBatchSize - // If there are no output columns, then the operator is for a COUNT query, - // in which case we treat the output batch size as the max int. - if o.output.Width() == 0 { - o.outputBatchSize = math.MaxInt64 - } - o.proberState.lBufferedGroup.spillingQueue = newSpillingQueue( o.unlimitedAllocator, o.left.sourceTypes, o.memoryLimit, - o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, + o.diskQueueCfg, o.fdSemaphore, o.diskAcc, ) o.proberState.lBufferedGroup.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( o.left.sourceTypes, 1, /* capacity */ ).ColVecs() o.proberState.rBufferedGroup.spillingQueue = newRewindableSpillingQueue( o.unlimitedAllocator, o.right.sourceTypes, o.memoryLimit, - o.diskQueueCfg, o.fdSemaphore, coldata.BatchSize(), o.diskAcc, + o.diskQueueCfg, o.fdSemaphore, o.diskAcc, ) o.proberState.rBufferedGroup.firstTuple = o.unlimitedAllocator.NewMemBatchWithFixedCapacity( o.right.sourceTypes, 1, /* capacity */ diff --git a/pkg/sql/colexec/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/mergejoiner_exceptall.eg.go index 3dc6dbf110c7..0952c250ae85 100644 --- a/pkg/sql/colexec/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/mergejoiner_exceptall.eg.go @@ -35511,7 +35511,6 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -35560,8 +35559,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35625,8 +35624,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35690,8 +35689,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35754,8 +35753,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35815,8 +35814,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35877,8 +35876,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35942,8 +35941,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36007,8 +36006,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36072,8 +36071,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36137,8 +36136,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36208,8 +36207,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36270,8 +36269,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36332,8 +36331,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36393,8 +36392,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36451,8 +36450,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36510,8 +36509,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36572,8 +36571,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36634,8 +36633,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36696,8 +36695,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36758,8 +36757,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36827,8 +36826,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36891,8 +36890,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36955,8 +36954,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37018,8 +37017,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37078,8 +37077,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37139,8 +37138,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37203,8 +37202,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37267,8 +37266,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37331,8 +37330,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37395,8 +37394,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37465,8 +37464,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37526,8 +37525,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37587,8 +37586,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37647,8 +37646,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37704,8 +37703,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37762,8 +37761,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37823,8 +37822,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37884,8 +37883,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37945,8 +37944,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38006,8 +38005,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38108,8 +38107,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38172,8 +38171,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38236,8 +38235,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38299,8 +38298,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38359,8 +38358,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38420,8 +38419,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38484,8 +38483,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38548,8 +38547,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38612,8 +38611,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38676,8 +38675,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38787,8 +38786,6 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -38825,8 +38822,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38893,8 +38890,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38961,8 +38958,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39028,8 +39025,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39092,8 +39089,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39157,8 +39154,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39225,8 +39222,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39293,8 +39290,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39361,8 +39358,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39429,8 +39426,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39503,8 +39500,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39569,8 +39566,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39635,8 +39632,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39700,8 +39697,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39762,8 +39759,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39825,8 +39822,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39891,8 +39888,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39957,8 +39954,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40023,8 +40020,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40089,8 +40086,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40163,8 +40160,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40231,8 +40228,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40299,8 +40296,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40366,8 +40363,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40430,8 +40427,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40495,8 +40492,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40563,8 +40560,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40631,8 +40628,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40699,8 +40696,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40767,8 +40764,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40841,8 +40838,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40907,8 +40904,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -40973,8 +40970,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41038,8 +41035,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41100,8 +41097,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41163,8 +41160,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41229,8 +41226,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41295,8 +41292,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41361,8 +41358,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41427,8 +41424,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -41521,8 +41518,8 @@ func (o *mergeJoinExceptAllOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -41947,9 +41944,9 @@ func (o *mergeJoinExceptAllOp) calculateOutputCount(groups []group) int { } count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -42034,7 +42031,7 @@ func (o *mergeJoinExceptAllOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/mergejoiner_fullouter.eg.go index 20042e0c6551..3b23d91904c1 100644 --- a/pkg/sql/colexec/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_fullouter.eg.go @@ -41695,7 +41695,6 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -41741,8 +41740,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -41806,8 +41805,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -41871,8 +41870,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -41935,8 +41934,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -41996,8 +41995,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42058,8 +42057,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42123,8 +42122,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42188,8 +42187,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42253,8 +42252,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42318,8 +42317,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42389,8 +42388,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42451,8 +42450,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42513,8 +42512,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42574,8 +42573,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42632,8 +42631,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42691,8 +42690,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42753,8 +42752,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42815,8 +42814,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42877,8 +42876,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -42939,8 +42938,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43008,8 +43007,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43072,8 +43071,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43136,8 +43135,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43199,8 +43198,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43259,8 +43258,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43320,8 +43319,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43384,8 +43383,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43448,8 +43447,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43512,8 +43511,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43576,8 +43575,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43646,8 +43645,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43707,8 +43706,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43768,8 +43767,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43828,8 +43827,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43885,8 +43884,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -43943,8 +43942,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -44004,8 +44003,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -44065,8 +44064,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -44126,8 +44125,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -44187,8 +44186,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -44291,8 +44290,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44340,8 +44339,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44389,8 +44388,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44437,8 +44436,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44482,8 +44481,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44528,8 +44527,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44577,8 +44576,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44626,8 +44625,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44675,8 +44674,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44724,8 +44723,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44821,8 +44820,6 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -44859,8 +44856,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -44929,8 +44926,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -44999,8 +44996,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45068,8 +45065,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45134,8 +45131,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45201,8 +45198,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45271,8 +45268,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45341,8 +45338,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45411,8 +45408,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45481,8 +45478,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45557,8 +45554,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45625,8 +45622,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45693,8 +45690,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45760,8 +45757,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45824,8 +45821,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45889,8 +45886,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -45957,8 +45954,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46025,8 +46022,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46093,8 +46090,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46161,8 +46158,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46237,8 +46234,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46307,8 +46304,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46377,8 +46374,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46446,8 +46443,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46512,8 +46509,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46579,8 +46576,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46649,8 +46646,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46719,8 +46716,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46789,8 +46786,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46859,8 +46856,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -46935,8 +46932,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47003,8 +47000,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47071,8 +47068,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47138,8 +47135,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47202,8 +47199,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47267,8 +47264,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47335,8 +47332,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47403,8 +47400,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47471,8 +47468,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47539,8 +47536,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -47635,8 +47632,8 @@ func (o *mergeJoinFullOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -48080,9 +48077,9 @@ func (o *mergeJoinFullOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -48180,7 +48177,7 @@ func (o *mergeJoinFullOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_inner.eg.go b/pkg/sql/colexec/mergejoiner_inner.eg.go index 18a07111d9ff..e8009e037a06 100644 --- a/pkg/sql/colexec/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/mergejoiner_inner.eg.go @@ -26047,7 +26047,6 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -26093,8 +26092,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26155,8 +26154,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26217,8 +26216,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26278,8 +26277,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26336,8 +26335,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26395,8 +26394,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26457,8 +26456,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26519,8 +26518,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26581,8 +26580,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26643,8 +26642,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26711,8 +26710,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26770,8 +26769,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26829,8 +26828,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26887,8 +26886,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26942,8 +26941,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26998,8 +26997,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27057,8 +27056,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27116,8 +27115,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27175,8 +27174,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27234,8 +27233,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27300,8 +27299,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27361,8 +27360,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27422,8 +27421,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27482,8 +27481,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27539,8 +27538,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27597,8 +27596,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27658,8 +27657,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27719,8 +27718,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27780,8 +27779,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27841,8 +27840,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27908,8 +27907,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27966,8 +27965,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28024,8 +28023,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28081,8 +28080,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28135,8 +28134,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28190,8 +28189,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28248,8 +28247,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28306,8 +28305,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28364,8 +28363,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28422,8 +28421,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28523,8 +28522,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28572,8 +28571,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28621,8 +28620,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28669,8 +28668,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28714,8 +28713,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28760,8 +28759,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28809,8 +28808,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28858,8 +28857,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28907,8 +28906,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28956,8 +28955,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -29053,8 +29052,6 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -29091,8 +29088,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29159,8 +29156,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29227,8 +29224,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29294,8 +29291,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29358,8 +29355,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29423,8 +29420,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29491,8 +29488,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29559,8 +29556,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29627,8 +29624,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29695,8 +29692,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29769,8 +29766,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29835,8 +29832,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29901,8 +29898,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29966,8 +29963,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30028,8 +30025,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30091,8 +30088,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30157,8 +30154,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30223,8 +30220,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30289,8 +30286,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30355,8 +30352,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30429,8 +30426,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30497,8 +30494,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30565,8 +30562,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30632,8 +30629,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30696,8 +30693,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30761,8 +30758,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30829,8 +30826,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30897,8 +30894,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30965,8 +30962,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31033,8 +31030,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31107,8 +31104,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31173,8 +31170,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31239,8 +31236,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31304,8 +31301,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31366,8 +31363,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31429,8 +31426,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31495,8 +31492,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31561,8 +31558,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31627,8 +31624,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31693,8 +31690,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31787,8 +31784,8 @@ func (o *mergeJoinInnerOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -32194,9 +32191,9 @@ func (o *mergeJoinInnerOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -32267,7 +32264,7 @@ func (o *mergeJoinInnerOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/mergejoiner_intersectall.eg.go index 52fe21af252b..140fe02407f0 100644 --- a/pkg/sql/colexec/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/mergejoiner_intersectall.eg.go @@ -27047,7 +27047,6 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -27093,8 +27092,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27155,8 +27154,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27217,8 +27216,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27278,8 +27277,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27336,8 +27335,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27395,8 +27394,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27457,8 +27456,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27519,8 +27518,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27581,8 +27580,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27643,8 +27642,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27711,8 +27710,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27770,8 +27769,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27829,8 +27828,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27887,8 +27886,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27942,8 +27941,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27998,8 +27997,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28057,8 +28056,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28116,8 +28115,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28175,8 +28174,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28234,8 +28233,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28300,8 +28299,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28361,8 +28360,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28422,8 +28421,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28482,8 +28481,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28539,8 +28538,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28597,8 +28596,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28658,8 +28657,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28719,8 +28718,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28780,8 +28779,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28841,8 +28840,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28908,8 +28907,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28966,8 +28965,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29024,8 +29023,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29081,8 +29080,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29135,8 +29134,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29190,8 +29189,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29248,8 +29247,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29306,8 +29305,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29364,8 +29363,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29422,8 +29421,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29524,8 +29523,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29588,8 +29587,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29652,8 +29651,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29715,8 +29714,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29775,8 +29774,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29836,8 +29835,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29900,8 +29899,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29964,8 +29963,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30028,8 +30027,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30092,8 +30091,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30203,8 +30202,6 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -30241,8 +30238,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30309,8 +30306,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30377,8 +30374,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30444,8 +30441,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30508,8 +30505,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30573,8 +30570,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30641,8 +30638,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30709,8 +30706,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30777,8 +30774,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30845,8 +30842,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30919,8 +30916,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30985,8 +30982,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31051,8 +31048,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31116,8 +31113,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31178,8 +31175,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31241,8 +31238,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31307,8 +31304,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31373,8 +31370,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31439,8 +31436,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31505,8 +31502,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31579,8 +31576,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31647,8 +31644,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31715,8 +31712,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31782,8 +31779,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31846,8 +31843,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31911,8 +31908,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31979,8 +31976,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32047,8 +32044,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32115,8 +32112,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32183,8 +32180,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32257,8 +32254,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32323,8 +32320,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32389,8 +32386,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32454,8 +32451,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32516,8 +32513,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32579,8 +32576,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32645,8 +32642,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32711,8 +32708,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32777,8 +32774,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32843,8 +32840,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -32937,8 +32934,8 @@ func (o *mergeJoinIntersectAllOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -33344,9 +33341,9 @@ func (o *mergeJoinIntersectAllOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -33415,7 +33412,7 @@ func (o *mergeJoinIntersectAllOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/mergejoiner_leftanti.eg.go index b86af80ca214..b8c89197ec8e 100644 --- a/pkg/sql/colexec/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftanti.eg.go @@ -33871,7 +33871,6 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -33920,8 +33919,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -33985,8 +33984,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34050,8 +34049,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34114,8 +34113,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34175,8 +34174,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34237,8 +34236,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34302,8 +34301,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34367,8 +34366,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34432,8 +34431,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34497,8 +34496,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34568,8 +34567,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34630,8 +34629,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34692,8 +34691,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34753,8 +34752,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34811,8 +34810,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34870,8 +34869,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34932,8 +34931,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34994,8 +34993,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35056,8 +35055,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35118,8 +35117,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35187,8 +35186,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35251,8 +35250,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35315,8 +35314,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35378,8 +35377,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35438,8 +35437,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35499,8 +35498,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35563,8 +35562,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35627,8 +35626,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35691,8 +35690,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35755,8 +35754,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35825,8 +35824,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35886,8 +35885,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35947,8 +35946,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36007,8 +36006,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36064,8 +36063,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36122,8 +36121,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36183,8 +36182,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36244,8 +36243,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36305,8 +36304,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36366,8 +36365,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36467,8 +36466,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36516,8 +36515,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36565,8 +36564,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36613,8 +36612,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36658,8 +36657,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36704,8 +36703,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36753,8 +36752,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36802,8 +36801,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36851,8 +36850,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36900,8 +36899,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36997,8 +36996,6 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -37035,8 +37032,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37103,8 +37100,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37171,8 +37168,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37238,8 +37235,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37302,8 +37299,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37367,8 +37364,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37435,8 +37432,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37503,8 +37500,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37571,8 +37568,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37639,8 +37636,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37713,8 +37710,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37779,8 +37776,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37845,8 +37842,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37910,8 +37907,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37972,8 +37969,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38035,8 +38032,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38101,8 +38098,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38167,8 +38164,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38233,8 +38230,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38299,8 +38296,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38373,8 +38370,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38441,8 +38438,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38509,8 +38506,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38576,8 +38573,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38640,8 +38637,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38705,8 +38702,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38773,8 +38770,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38841,8 +38838,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38909,8 +38906,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38977,8 +38974,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39051,8 +39048,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39117,8 +39114,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39183,8 +39180,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39248,8 +39245,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39310,8 +39307,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39373,8 +39370,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39439,8 +39436,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39505,8 +39502,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39571,8 +39568,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39637,8 +39634,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39731,8 +39728,8 @@ func (o *mergeJoinLeftAntiOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -40141,9 +40138,9 @@ func (o *mergeJoinLeftAntiOp) calculateOutputCount(groups []group) int { } count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -40228,7 +40225,7 @@ func (o *mergeJoinLeftAntiOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/mergejoiner_leftouter.eg.go index f1fe95ea2530..108d595ee4d8 100644 --- a/pkg/sql/colexec/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftouter.eg.go @@ -34031,7 +34031,6 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -34077,8 +34076,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34139,8 +34138,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34201,8 +34200,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34262,8 +34261,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34320,8 +34319,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34379,8 +34378,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34441,8 +34440,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34503,8 +34502,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34565,8 +34564,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34627,8 +34626,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34695,8 +34694,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34754,8 +34753,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34813,8 +34812,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34871,8 +34870,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34926,8 +34925,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -34982,8 +34981,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35041,8 +35040,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35100,8 +35099,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35159,8 +35158,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35218,8 +35217,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35284,8 +35283,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35345,8 +35344,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35406,8 +35405,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35466,8 +35465,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35523,8 +35522,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35581,8 +35580,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35642,8 +35641,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35703,8 +35702,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35764,8 +35763,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35825,8 +35824,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35892,8 +35891,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -35950,8 +35949,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36008,8 +36007,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36065,8 +36064,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36119,8 +36118,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36174,8 +36173,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36232,8 +36231,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36290,8 +36289,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36348,8 +36347,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36406,8 +36405,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36507,8 +36506,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36556,8 +36555,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36605,8 +36604,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36653,8 +36652,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36698,8 +36697,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36744,8 +36743,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36793,8 +36792,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36842,8 +36841,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36891,8 +36890,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36940,8 +36939,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -37037,8 +37036,6 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -37075,8 +37072,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37145,8 +37142,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37215,8 +37212,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37284,8 +37281,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37350,8 +37347,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37417,8 +37414,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37487,8 +37484,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37557,8 +37554,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37627,8 +37624,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37697,8 +37694,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37773,8 +37770,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37841,8 +37838,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37909,8 +37906,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -37976,8 +37973,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38040,8 +38037,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38105,8 +38102,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38173,8 +38170,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38241,8 +38238,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38309,8 +38306,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38377,8 +38374,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38453,8 +38450,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38523,8 +38520,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38593,8 +38590,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38662,8 +38659,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38728,8 +38725,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38795,8 +38792,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38865,8 +38862,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -38935,8 +38932,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39005,8 +39002,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39075,8 +39072,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39151,8 +39148,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39219,8 +39216,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39287,8 +39284,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39354,8 +39351,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39418,8 +39415,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39483,8 +39480,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39551,8 +39548,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39619,8 +39616,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39687,8 +39684,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39755,8 +39752,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if rightGroup.nullGroup { @@ -39851,8 +39848,8 @@ func (o *mergeJoinLeftOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -40278,9 +40275,9 @@ func (o *mergeJoinLeftOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -40367,7 +40364,7 @@ func (o *mergeJoinLeftOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go index 577b8202fc49..5fc484f408cb 100644 --- a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go @@ -25887,7 +25887,6 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -25933,8 +25932,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -25995,8 +25994,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26057,8 +26056,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26118,8 +26117,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26176,8 +26175,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26235,8 +26234,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26297,8 +26296,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26359,8 +26358,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26421,8 +26420,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26483,8 +26482,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26551,8 +26550,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26610,8 +26609,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26669,8 +26668,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26727,8 +26726,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26782,8 +26781,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26838,8 +26837,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26897,8 +26896,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -26956,8 +26955,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27015,8 +27014,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27074,8 +27073,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27140,8 +27139,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27201,8 +27200,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27262,8 +27261,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27322,8 +27321,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27379,8 +27378,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27437,8 +27436,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27498,8 +27497,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27559,8 +27558,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27620,8 +27619,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27681,8 +27680,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27748,8 +27747,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27806,8 +27805,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27864,8 +27863,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27921,8 +27920,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -27975,8 +27974,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28030,8 +28029,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28088,8 +28087,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28146,8 +28145,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28204,8 +28203,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28262,8 +28261,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28363,8 +28362,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28412,8 +28411,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28461,8 +28460,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28509,8 +28508,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28554,8 +28553,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28600,8 +28599,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28649,8 +28648,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28698,8 +28697,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28747,8 +28746,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28796,8 +28795,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28893,8 +28892,6 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -28931,8 +28928,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -28999,8 +28996,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29067,8 +29064,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29134,8 +29131,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29198,8 +29195,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29263,8 +29260,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29331,8 +29328,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29399,8 +29396,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29467,8 +29464,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29535,8 +29532,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29609,8 +29606,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29675,8 +29672,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29741,8 +29738,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29806,8 +29803,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29868,8 +29865,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29931,8 +29928,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -29997,8 +29994,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30063,8 +30060,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30129,8 +30126,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30195,8 +30192,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30269,8 +30266,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30337,8 +30334,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30405,8 +30402,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30472,8 +30469,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30536,8 +30533,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30601,8 +30598,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30669,8 +30666,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30737,8 +30734,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30805,8 +30802,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30873,8 +30870,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -30947,8 +30944,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31013,8 +31010,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31079,8 +31076,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31144,8 +31141,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31206,8 +31203,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31269,8 +31266,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31335,8 +31332,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31401,8 +31398,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31467,8 +31464,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31533,8 +31530,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -31627,8 +31624,8 @@ func (o *mergeJoinLeftSemiOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -32031,9 +32028,9 @@ func (o *mergeJoinLeftSemiOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -32102,7 +32099,7 @@ func (o *mergeJoinLeftSemiOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/mergejoiner_rightouter.eg.go index 9de9918d6b40..523ddf4934a5 100644 --- a/pkg/sql/colexec/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_rightouter.eg.go @@ -33711,7 +33711,6 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -33757,8 +33756,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -33822,8 +33821,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -33887,8 +33886,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -33951,8 +33950,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34012,8 +34011,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34074,8 +34073,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34139,8 +34138,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34204,8 +34203,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34269,8 +34268,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34334,8 +34333,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34405,8 +34404,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34467,8 +34466,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34529,8 +34528,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34590,8 +34589,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34648,8 +34647,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34707,8 +34706,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34769,8 +34768,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34831,8 +34830,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34893,8 +34892,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -34955,8 +34954,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35024,8 +35023,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35088,8 +35087,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35152,8 +35151,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35215,8 +35214,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35275,8 +35274,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35336,8 +35335,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35400,8 +35399,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35464,8 +35463,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35528,8 +35527,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35592,8 +35591,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35662,8 +35661,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35723,8 +35722,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35784,8 +35783,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35844,8 +35843,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35901,8 +35900,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -35959,8 +35958,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -36020,8 +36019,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -36081,8 +36080,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -36142,8 +36141,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -36203,8 +36202,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if leftGroup.nullGroup { @@ -36307,8 +36306,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36356,8 +36355,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36405,8 +36404,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36453,8 +36452,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36498,8 +36497,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36544,8 +36543,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36593,8 +36592,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36642,8 +36641,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36691,8 +36690,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36740,8 +36739,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36837,8 +36836,6 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -36875,8 +36872,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -36943,8 +36940,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37011,8 +37008,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37078,8 +37075,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37142,8 +37139,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37207,8 +37204,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37275,8 +37272,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37343,8 +37340,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37411,8 +37408,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37479,8 +37476,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37553,8 +37550,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37619,8 +37616,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37685,8 +37682,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37750,8 +37747,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37812,8 +37809,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37875,8 +37872,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -37941,8 +37938,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38007,8 +38004,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38073,8 +38070,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38139,8 +38136,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38213,8 +38210,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38281,8 +38278,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38349,8 +38346,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38416,8 +38413,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38480,8 +38477,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38545,8 +38542,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38613,8 +38610,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38681,8 +38678,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38749,8 +38746,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38817,8 +38814,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38891,8 +38888,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -38957,8 +38954,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39023,8 +39020,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39088,8 +39085,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39150,8 +39147,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39213,8 +39210,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39279,8 +39276,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39345,8 +39342,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39411,8 +39408,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39477,8 +39474,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } { @@ -39571,8 +39568,8 @@ func (o *mergeJoinRightOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -39996,9 +39993,9 @@ func (o *mergeJoinRightOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -40085,7 +40082,7 @@ func (o *mergeJoinRightOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_test.go b/pkg/sql/colexec/mergejoiner_test.go index 78de22c9fde0..0df0f56dc745 100644 --- a/pkg/sql/colexec/mergejoiner_test.go +++ b/pkg/sql/colexec/mergejoiner_test.go @@ -217,7 +217,7 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1}, {1}, {2}, {2}, {2}, {3}, {4}}, }, { - description: "cross product test, batch size = col.BatchSize()", + description: "cross product test", leftTypes: []*types.T{types.Int}, rightTypes: []*types.T{types.Int}, leftTuples: tuples{{1}, {1}, {1}, {1}}, @@ -229,46 +229,7 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, }, { - description: "cross product test, batch size = 4 (small even)", - leftTypes: []*types.T{types.Int}, - rightTypes: []*types.T{types.Int}, - leftTuples: tuples{{1}, {1}, {1}, {1}}, - rightTuples: tuples{{1}, {1}, {1}, {1}}, - leftOutCols: []uint32{0}, - rightOutCols: []uint32{}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, - outputBatchSize: 4, - }, - { - description: "cross product test, batch size = 3 (small odd)", - leftTypes: []*types.T{types.Int}, - rightTypes: []*types.T{types.Int}, - leftTuples: tuples{{1}, {1}, {1}, {1}}, - rightTuples: tuples{{1}, {1}, {1}, {1}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, - outputBatchSize: 3, - }, - { - description: "cross product test, batch size = 1 (unit)", - leftTypes: []*types.T{types.Int}, - rightTypes: []*types.T{types.Int}, - leftTuples: tuples{{1}, {1}, {1}, {1}}, - rightTuples: tuples{{1}, {1}, {1}, {1}}, - leftOutCols: []uint32{}, - rightOutCols: []uint32{0}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}, {1}}, - outputBatchSize: 1, - }, - { - description: "multi output column test, basic", + description: "multi output column test", leftTypes: []*types.T{types.Int, types.Int}, rightTypes: []*types.T{types.Int, types.Int}, leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, @@ -279,19 +240,6 @@ var mjTestCases = []*joinTestCase{ rightEqCols: []uint32{0}, expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, - { - description: "multi output column test, batch size = 1", - leftTypes: []*types.T{types.Int, types.Int}, - rightTypes: []*types.T{types.Int, types.Int}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, - outputBatchSize: 1, - }, { description: "multi output column test, test output coldata projection", leftTypes: []*types.T{types.Int, types.Int}, @@ -328,19 +276,6 @@ var mjTestCases = []*joinTestCase{ rightEqCols: []uint32{0}, expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {2, 21, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, - { - description: "multi output column test, L run, batch size = 1", - leftTypes: []*types.T{types.Int, types.Int}, - rightTypes: []*types.T{types.Int, types.Int}, - leftTuples: tuples{{1, 10}, {2, 20}, {2, 21}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {2, 20, 2, 12}, {2, 21, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, - outputBatchSize: 1, - }, { description: "multi output column test, R run", leftTypes: []*types.T{types.Int, types.Int}, @@ -353,19 +288,6 @@ var mjTestCases = []*joinTestCase{ rightEqCols: []uint32{0}, expected: tuples{{1, 10, 1, 11}, {1, 10, 1, 111}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, }, - { - description: "multi output column test, R run, batch size = 1", - leftTypes: []*types.T{types.Int, types.Int}, - rightTypes: []*types.T{types.Int, types.Int}, - leftTuples: tuples{{1, 10}, {2, 20}, {3, 30}, {4, 40}}, - rightTuples: tuples{{1, 11}, {1, 111}, {2, 12}, {3, 13}, {4, 14}}, - leftOutCols: []uint32{0, 1}, - rightOutCols: []uint32{0, 1}, - leftEqCols: []uint32{0}, - rightEqCols: []uint32{0}, - expected: tuples{{1, 10, 1, 11}, {1, 10, 1, 111}, {2, 20, 2, 12}, {3, 30, 3, 13}, {4, 40, 4, 14}}, - outputBatchSize: 1, - }, { description: "logic test", leftTypes: []*types.T{types.Int, types.Int}, @@ -379,7 +301,7 @@ var mjTestCases = []*joinTestCase{ expected: tuples{{4, 5}, {4, 2}}, }, { - description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns", + description: "multi output column test, runs (to test saved output), reordered out columns", leftTypes: []*types.T{types.Int, types.Int}, rightTypes: []*types.T{types.Int, types.Int}, leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, @@ -399,10 +321,9 @@ var mjTestCases = []*joinTestCase{ {30, 3, 13, 3}, {40, 4, 14, 4}, }, - outputBatchSize: 1, }, { - description: "multi output column test, batch size = 1 and runs (to test saved output), reordered out columns that dont start at 0", + description: "multi output column test, runs (to test saved output), reordered out columns that dont start at 0", leftTypes: []*types.T{types.Int, types.Int}, rightTypes: []*types.T{types.Int, types.Int}, leftTuples: tuples{{1, 10}, {1, 10}, {1, 10}, {2, 20}, {3, 30}, {4, 40}}, @@ -422,7 +343,6 @@ var mjTestCases = []*joinTestCase{ {30, 3, 13}, {40, 4, 14}, }, - outputBatchSize: 1, }, { description: "equality column is correctly indexed", @@ -576,40 +496,6 @@ var mjTestCases = []*joinTestCase{ {true, 20, 2.2, true, 20, 2.4}, }, }, - { - description: "templating cross product test, output batch size 1", - leftTypes: []*types.T{types.Bool, types.Int2, types.Float}, - rightTypes: []*types.T{types.Bool, types.Int2, types.Float}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{ - {false, 10, 1.2, false, 10, 1.2}, - {true, 20, 2.2, true, 20, 2.3}, - {true, 20, 2.2, true, 20, 2.4}, - }, - outputBatchSize: 1, - }, - { - description: "templating cross product test, output batch size 2", - leftTypes: []*types.T{types.Bool, types.Int2, types.Float}, - rightTypes: []*types.T{types.Bool, types.Int2, types.Float}, - leftTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.2}, {true, int16(30), 3.2}}, - rightTuples: tuples{{false, int16(10), 1.2}, {true, int16(20), 2.3}, {true, int16(20), 2.4}, {true, int16(31), 3.9}}, - leftOutCols: []uint32{0, 1, 2}, - rightOutCols: []uint32{0, 1, 2}, - leftEqCols: []uint32{0, 1}, - rightEqCols: []uint32{0, 1}, - expected: tuples{ - {false, 10, 1.2, false, 10, 1.2}, - {true, 20, 2.2, true, 20, 2.3}, - {true, 20, 2.2, true, 20, 2.4}, - }, - outputBatchSize: 2, - }, { description: "templating reordered eq columns", leftTypes: []*types.T{types.Bool, types.Int2, types.Float}, @@ -1659,28 +1545,13 @@ func TestMergeJoiner(t *testing.T) { for _, tc := range mjTestCases { for _, tc := range tc.mutateTypes() { tc.init() - - // We use a custom verifier function so that we can get the merge join op - // to use a custom output batch size per test, to exercise more cases. - var mergeJoinVerifier verifierFn = func(output *opTestOutput) error { - if mj, ok := output.input.(variableOutputBatchSizeInitializer); ok { - mj.initWithOutputBatchSize(tc.outputBatchSize) - } else { - // When we have an inner join with ON expression, a filter operator - // will be put on top of the merge join, so to make life easier, we'll - // just ignore the requested output batch size. - output.input.Init() - } - verify := output.Verify - if tc.joinType == descpb.FullOuterJoin { - // FULL OUTER JOIN doesn't guarantee any ordering on its output (since - // it is ambiguous), so we're comparing the outputs as sets. - verify = output.VerifyAnyOrder - } - - return verify() + verifier := orderedVerifier + if tc.joinType == descpb.FullOuterJoin { + // FULL OUTER JOIN doesn't guarantee any ordering on its output + // (since it is ambiguous), so we're comparing the outputs as + // sets. + verifier = unorderedVerifier } - var runner testRunner if tc.skipAllNullsInjection { // We're omitting all nulls injection test. See comments for each such @@ -1695,7 +1566,7 @@ func TestMergeJoiner(t *testing.T) { log.Infof(context.Background(), "MemoryLimit=%s/%s", humanizeutil.IBytes(memoryLimit), tc.description) runner(t, []tuples{tc.leftTuples, tc.rightTuples}, [][]*types.T{tc.leftTypes, tc.rightTypes}, - tc.expected, mergeJoinVerifier, + tc.expected, verifier, func(input []colexecbase.Operator) (colexecbase.Operator, error) { spec := createSpecForMergeJoiner(tc) args := &NewColOperatorArgs{ @@ -1742,69 +1613,64 @@ func TestFullOuterMergeJoinWithMaximumNumberOfGroups(t *testing.T) { nTuples := coldata.BatchSize() * 4 queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() - for _, outBatchSize := range []int{1, 16, coldata.BatchSize() - 1, coldata.BatchSize(), coldata.BatchSize() + 1} { - t.Run(fmt.Sprintf("outBatchSize=%d", outBatchSize), - func(t *testing.T) { - typs := []*types.T{types.Int} - colsLeft := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} - colsRight := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} - groupsLeft := colsLeft[0].Int64() - groupsRight := colsRight[0].Int64() - for i := range groupsLeft { - groupsLeft[i] = int64(i * 2) - groupsRight[i] = int64(i*2 + 1) - } - leftSource := newChunkingBatchSource(typs, colsLeft, nTuples) - rightSource := newChunkingBatchSource(typs, colsRight, nTuples) - a, err := NewMergeJoinOp( - testAllocator, defaultMemoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(mjFDLimit), descpb.FullOuterJoin, - leftSource, rightSource, typs, typs, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - testDiskAcc, - ) - if err != nil { - t.Fatal("error in merge join op constructor", err) - } - a.(*mergeJoinFullOuterOp).initWithOutputBatchSize(outBatchSize) - i, count, expVal := 0, 0, int64(0) - for b := a.Next(ctx); b.Length() != 0; b = a.Next(ctx) { - count += b.Length() - leftOutCol := b.ColVec(0).Int64() - leftNulls := b.ColVec(0).Nulls() - rightOutCol := b.ColVec(1).Int64() - rightNulls := b.ColVec(1).Nulls() - for j := 0; j < b.Length(); j++ { - leftVal := leftOutCol[j] - leftNull := leftNulls.NullAt(j) - rightVal := rightOutCol[j] - rightNull := rightNulls.NullAt(j) - if expVal%2 == 0 { - // It is an even-numbered row, so the left value should contain - // expVal and the right value should be NULL. - if leftVal != expVal || leftNull || !rightNull { - t.Fatalf("found left = %d, left NULL? = %t, right NULL? = %t, "+ - "expected left = %d, left NULL? = false, right NULL? = true, idx %d of batch %d", - leftVal, leftNull, rightNull, expVal, j, i) - } - } else { - // It is an odd-numbered row, so the right value should contain - // expVal and the left value should be NULL. - if rightVal != expVal || rightNull || !leftNull { - t.Fatalf("found right = %d, right NULL? = %t, left NULL? = %t, "+ - "expected right = %d, right NULL? = false, left NULL? = true, idx %d of batch %d", - rightVal, rightNull, leftNull, expVal, j, i) - } - } - expVal++ - } - i++ + typs := []*types.T{types.Int} + colsLeft := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + colsRight := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + groupsLeft := colsLeft[0].Int64() + groupsRight := colsRight[0].Int64() + for i := range groupsLeft { + groupsLeft[i] = int64(i * 2) + groupsRight[i] = int64(i*2 + 1) + } + leftSource := newChunkingBatchSource(typs, colsLeft, nTuples) + rightSource := newChunkingBatchSource(typs, colsRight, nTuples) + a, err := NewMergeJoinOp( + testAllocator, defaultMemoryLimit, queueCfg, + colexecbase.NewTestingSemaphore(mjFDLimit), descpb.FullOuterJoin, + leftSource, rightSource, typs, typs, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + testDiskAcc, + ) + if err != nil { + t.Fatal("error in merge join op constructor", err) + } + a.Init() + i, count, expVal := 0, 0, int64(0) + for b := a.Next(ctx); b.Length() != 0; b = a.Next(ctx) { + count += b.Length() + leftOutCol := b.ColVec(0).Int64() + leftNulls := b.ColVec(0).Nulls() + rightOutCol := b.ColVec(1).Int64() + rightNulls := b.ColVec(1).Nulls() + for j := 0; j < b.Length(); j++ { + leftVal := leftOutCol[j] + leftNull := leftNulls.NullAt(j) + rightVal := rightOutCol[j] + rightNull := rightNulls.NullAt(j) + if expVal%2 == 0 { + // It is an even-numbered row, so the left value should contain + // expVal and the right value should be NULL. + if leftVal != expVal || leftNull || !rightNull { + t.Fatalf("found left = %d, left NULL? = %t, right NULL? = %t, "+ + "expected left = %d, left NULL? = false, right NULL? = true, idx %d of batch %d", + leftVal, leftNull, rightNull, expVal, j, i) } - if count != 2*nTuples { - t.Fatalf("found count %d, expected count %d", count, 2*nTuples) + } else { + // It is an odd-numbered row, so the right value should contain + // expVal and the left value should be NULL. + if rightVal != expVal || rightNull || !leftNull { + t.Fatalf("found right = %d, right NULL? = %t, left NULL? = %t, "+ + "expected right = %d, right NULL? = false, left NULL? = true, idx %d of batch %d", + rightVal, rightNull, leftNull, expVal, j, i) } - }) + } + expVal++ + } + i++ + } + if count != 2*nTuples { + t.Fatalf("found count %d, expected count %d", count, 2*nTuples) } } @@ -1830,87 +1696,81 @@ func TestMergeJoinCrossProduct(t *testing.T) { queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() rng, _ := randutil.NewPseudoRand() - for _, outBatchSize := range []int{1, 17, coldata.BatchSize() - 1, coldata.BatchSize(), coldata.BatchSize() + 1} { - t.Run(fmt.Sprintf("outBatchSize=%d", outBatchSize), - func(t *testing.T) { - typs := []*types.T{types.Int, types.Bytes, types.Decimal} - colsLeft := make([]coldata.Vec, len(typs)) - colsRight := make([]coldata.Vec, len(typs)) - for i, typ := range typs { - colsLeft[i] = testAllocator.NewMemColumn(typ, nTuples) - colsRight[i] = testAllocator.NewMemColumn(typ, nTuples) - } - groupsLeft := colsLeft[0].Int64() - groupsRight := colsRight[0].Int64() - leftGroupIdx, rightGroupIdx := 0, 0 - for i := range groupsLeft { - if rng.Float64() < 1.0/float64(coldata.BatchSize()) { - leftGroupIdx++ - } - if rng.Float64() < 1.0/float64(coldata.BatchSize()) { - rightGroupIdx++ - } - groupsLeft[i] = int64(leftGroupIdx) - groupsRight[i] = int64(rightGroupIdx) - } - for i := range typs[1:] { - for _, vecs := range [][]coldata.Vec{colsLeft, colsRight} { - coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{ - Rand: rng, - Vec: vecs[i+1], - N: nTuples, - NullProbability: nullProbability, - }) - } - } - leftMJSource := newChunkingBatchSource(typs, colsLeft, nTuples) - rightMJSource := newChunkingBatchSource(typs, colsRight, nTuples) - leftHJSource := newChunkingBatchSource(typs, colsLeft, nTuples) - rightHJSource := newChunkingBatchSource(typs, colsRight, nTuples) - mj, err := NewMergeJoinOp( - testAllocator, defaultMemoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, - leftMJSource, rightMJSource, typs, typs, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - testDiskAcc, - ) - if err != nil { - t.Fatal("error in merge join op constructor", err) - } - mj.(*mergeJoinInnerOp).initWithOutputBatchSize(outBatchSize) - hj := NewHashJoiner( - testAllocator, HashJoinerSpec{ - joinType: descpb.InnerJoin, - left: hashJoinerSourceSpec{ - eqCols: []uint32{0}, sourceTypes: typs, - }, - right: hashJoinerSourceSpec{ - eqCols: []uint32{0}, sourceTypes: typs, - }, - }, leftHJSource, rightHJSource) - hj.Init() - - var mjOutputTuples, hjOutputTuples tuples - for b := mj.Next(ctx); b.Length() != 0; b = mj.Next(ctx) { - for i := 0; i < b.Length(); i++ { - mjOutputTuples = append(mjOutputTuples, getTupleFromBatch(b, i)) - } - } - for b := hj.Next(ctx); b.Length() != 0; b = hj.Next(ctx) { - for i := 0; i < b.Length(); i++ { - hjOutputTuples = append(hjOutputTuples, getTupleFromBatch(b, i)) - } - } - err = assertTuplesSetsEqual(hjOutputTuples, mjOutputTuples, evalCtx) - // Note that the error message can be extremely verbose (it - // might contain all output tuples), so we manually check that - // comparing err to nil returns true (if we were to use - // require.NoError, then the error message would be output). - require.True(t, err == nil) + typs := []*types.T{types.Int, types.Bytes, types.Decimal} + colsLeft := make([]coldata.Vec, len(typs)) + colsRight := make([]coldata.Vec, len(typs)) + for i, typ := range typs { + colsLeft[i] = testAllocator.NewMemColumn(typ, nTuples) + colsRight[i] = testAllocator.NewMemColumn(typ, nTuples) + } + groupsLeft := colsLeft[0].Int64() + groupsRight := colsRight[0].Int64() + leftGroupIdx, rightGroupIdx := 0, 0 + for i := range groupsLeft { + if rng.Float64() < 1.0/float64(coldata.BatchSize()) { + leftGroupIdx++ + } + if rng.Float64() < 1.0/float64(coldata.BatchSize()) { + rightGroupIdx++ + } + groupsLeft[i] = int64(leftGroupIdx) + groupsRight[i] = int64(rightGroupIdx) + } + for i := range typs[1:] { + for _, vecs := range [][]coldata.Vec{colsLeft, colsRight} { + coldatatestutils.RandomVec(coldatatestutils.RandomVecArgs{ + Rand: rng, + Vec: vecs[i+1], + N: nTuples, + NullProbability: nullProbability, }) + } + } + leftMJSource := newChunkingBatchSource(typs, colsLeft, nTuples) + rightMJSource := newChunkingBatchSource(typs, colsRight, nTuples) + leftHJSource := newChunkingBatchSource(typs, colsLeft, nTuples) + rightHJSource := newChunkingBatchSource(typs, colsRight, nTuples) + mj, err := NewMergeJoinOp( + testAllocator, defaultMemoryLimit, queueCfg, + colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + leftMJSource, rightMJSource, typs, typs, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + testDiskAcc, + ) + if err != nil { + t.Fatal("error in merge join op constructor", err) + } + mj.Init() + hj := NewHashJoiner( + testAllocator, HashJoinerSpec{ + joinType: descpb.InnerJoin, + left: hashJoinerSourceSpec{ + eqCols: []uint32{0}, sourceTypes: typs, + }, + right: hashJoinerSourceSpec{ + eqCols: []uint32{0}, sourceTypes: typs, + }, + }, leftHJSource, rightHJSource) + hj.Init() + var mjOutputTuples, hjOutputTuples tuples + for b := mj.Next(ctx); b.Length() != 0; b = mj.Next(ctx) { + for i := 0; i < b.Length(); i++ { + mjOutputTuples = append(mjOutputTuples, getTupleFromBatch(b, i)) + } + } + for b := hj.Next(ctx); b.Length() != 0; b = hj.Next(ctx) { + for i := 0; i < b.Length(); i++ { + hjOutputTuples = append(hjOutputTuples, getTupleFromBatch(b, i)) + } } + err = assertTuplesSetsEqual(hjOutputTuples, mjOutputTuples, evalCtx) + // Note that the error message can be extremely verbose (it + // might contain all output tuples), so we manually check that + // comparing err to nil returns true (if we were to use + // require.NoError, then the error message would be output). + require.True(t, err == nil) } // TestMergeJoinerMultiBatch creates one long input of a 1:1 join, and keeps @@ -1923,56 +1783,50 @@ func TestMergeJoinerMultiBatch(t *testing.T) { queueCfg, cleanup := colcontainerutils.NewTestingDiskQueueCfg(t, true /* inMem */) defer cleanup() for _, numInputBatches := range []int{1, 2, 16} { - for _, outBatchSize := range []int{1, 16, coldata.BatchSize()} { - t.Run(fmt.Sprintf("numInputBatches=%d", numInputBatches), - func(t *testing.T) { - nTuples := coldata.BatchSize() * numInputBatches - typs := []*types.T{types.Int} - cols := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} - groups := cols[0].Int64() - for i := range groups { - groups[i] = int64(i) - } - - leftSource := newChunkingBatchSource(typs, cols, nTuples) - rightSource := newChunkingBatchSource(typs, cols, nTuples) - - a, err := NewMergeJoinOp( - testAllocator, defaultMemoryLimit, - queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, - leftSource, rightSource, typs, typs, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, - testDiskAcc, - ) - if err != nil { - t.Fatal("error in merge join op constructor", err) - } - - a.(*mergeJoinInnerOp).initWithOutputBatchSize(outBatchSize) - - i := 0 - count := 0 - // Keep track of the last comparison value. - expVal := int64(0) - for b := a.Next(ctx); b.Length() != 0; b = a.Next(ctx) { - count += b.Length() - outCol := b.ColVec(0).Int64() - for j := int64(0); j < int64(b.Length()); j++ { - outVal := outCol[j] - if outVal != expVal { - t.Fatalf("found val %d, expected %d, idx %d of batch %d", - outVal, expVal, j, i) - } - expVal++ + t.Run(fmt.Sprintf("numInputBatches=%d", numInputBatches), + func(t *testing.T) { + nTuples := coldata.BatchSize() * numInputBatches + typs := []*types.T{types.Int} + cols := []coldata.Vec{testAllocator.NewMemColumn(typs[0], nTuples)} + groups := cols[0].Int64() + for i := range groups { + groups[i] = int64(i) + } + leftSource := newChunkingBatchSource(typs, cols, nTuples) + rightSource := newChunkingBatchSource(typs, cols, nTuples) + a, err := NewMergeJoinOp( + testAllocator, defaultMemoryLimit, + queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, + leftSource, rightSource, typs, typs, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, + testDiskAcc, + ) + if err != nil { + t.Fatal("error in merge join op constructor", err) + } + a.Init() + i := 0 + count := 0 + // Keep track of the last comparison value. + expVal := int64(0) + for b := a.Next(ctx); b.Length() != 0; b = a.Next(ctx) { + count += b.Length() + outCol := b.ColVec(0).Int64() + for j := int64(0); j < int64(b.Length()); j++ { + outVal := outCol[j] + if outVal != expVal { + t.Fatalf("found val %d, expected %d, idx %d of batch %d", + outVal, expVal, j, i) } - i++ - } - if count != nTuples { - t.Fatalf("found count %d, expected count %d", count, nTuples) + expVal++ } - }) - } + i++ + } + if count != nTuples { + t.Fatalf("found count %d, expected count %d", count, nTuples) + } + }) } } @@ -2014,10 +1868,8 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { cols[0].Int64()[i] = int64(i / groupSize) cols[1].Int64()[i] = int64(i / groupSize) } - leftSource := newChunkingBatchSource(typs, cols, nTuples) rightSource := newChunkingBatchSource(typs, cols, nTuples) - a, err := NewMergeJoinOp( testAllocator, defaultMemoryLimit, queueCfg, colexecbase.NewTestingSemaphore(mjFDLimit), descpb.InnerJoin, @@ -2029,9 +1881,7 @@ func TestMergeJoinerMultiBatchRuns(t *testing.T) { if err != nil { t.Fatal("error in merge join op constructor", err) } - - a.(*mergeJoinInnerOp).Init() - + a.Init() i := 0 count := 0 // Keep track of the last comparison value. @@ -2156,13 +2006,10 @@ func TestMergeJoinerRandomized(t *testing.T) { []execinfrapb.Ordering_Column{{ColIdx: 0, Direction: execinfrapb.Ordering_Column_ASC}}, testDiskAcc, ) - if err != nil { t.Fatal("error in merge join op constructor", err) } - - a.(*mergeJoinInnerOp).Init() - + a.Init() i := 0 count := 0 cpIdx := 0 diff --git a/pkg/sql/colexec/mergejoiner_tmpl.go b/pkg/sql/colexec/mergejoiner_tmpl.go index e76518244a96..2662a9732d33 100644 --- a/pkg/sql/colexec/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/mergejoiner_tmpl.go @@ -747,8 +747,8 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool) repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // {{if _JOIN_TYPE.IsRightOuter}} @@ -832,7 +832,6 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftGroupsFromBatch( ) { sel := batch.Selection() initialBuilderState := o.builderState.left - outputBatchSize := o.outputBatchSize o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[:len(input.sourceTypes)], func() { @@ -935,8 +934,8 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // {{if _JOIN_TYPE.IsSetOp}} @@ -1065,8 +1064,8 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > outputBatchSize { - toAppend = outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // {{if _JOIN_TYPE.IsLeftOuter}} @@ -1174,8 +1173,6 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightGroupsFromBatch( ) { initialBuilderState := o.builderState.right sel := batch.Selection() - outputBatchSize := o.outputBatchSize - o.unlimitedAllocator.PerformOperation( o.output.ColVecs()[colOffset:colOffset+len(input.sourceTypes)], func() { @@ -1244,8 +1241,8 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > o.outputBatchSize { - toAppend = o.outputBatchSize - outStartIdx + if outStartIdx+toAppend > coldata.BatchSize() { + toAppend = coldata.BatchSize() - outStartIdx } // Loop over every column. @@ -1509,9 +1506,9 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) calculateOutputCount(groups []group) int // {{end}} count += groups[i].toBuild groups[i].toBuild = 0 - if count > o.outputBatchSize { - groups[i].toBuild = count - o.outputBatchSize - count = o.outputBatchSize + if count > coldata.BatchSize() { + groups[i].toBuild = count - coldata.BatchSize() + count = coldata.BatchSize() return count } } @@ -1633,7 +1630,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == o.outputBatchSize { + if o.outputReady || o.builderState.outCount == coldata.BatchSize() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index b151651b5533..d4de2ca0cb9e 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -63,11 +63,6 @@ type orderedAggregator struct { // resumeIdx is the index at which the aggregation functions should start // writing to on the next iteration of Next(). resumeIdx int - // inputSize and outputSize are 2*coldata.BatchSize() and - // coldata.BatchSize(), respectively, by default but can be other values - // for tests. - inputSize int - outputSize int } // unsafeBatch is a coldata.Batch returned when only a subset of the @@ -159,27 +154,16 @@ func NewOrderedAggregator( return a, nil } -func (a *orderedAggregator) initWithOutputBatchSize(outputSize int) { - a.initWithInputAndOutputBatchSize(coldata.BatchSize(), outputSize) -} - -func (a *orderedAggregator) initWithInputAndOutputBatchSize(inputSize, outputSize int) { +func (a *orderedAggregator) Init() { a.input.Init() - - // Twice the input batchSize is allocated to avoid having to check for - // overflow when outputting. - a.scratch.inputSize = inputSize * 2 - a.scratch.outputSize = outputSize - a.scratch.Batch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, a.scratch.inputSize) + // Twice the batchSize is allocated to avoid having to check for overflow + // when outputting. + a.scratch.Batch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, 2*coldata.BatchSize()) for i := 0; i < len(a.outputTypes); i++ { vec := a.scratch.ColVec(i) a.aggregateFuncs[i].Init(a.groupCol, vec) } - a.unsafeBatch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, outputSize) -} - -func (a *orderedAggregator) Init() { - a.initWithInputAndOutputBatchSize(coldata.BatchSize(), coldata.BatchSize()) + a.unsafeBatch = a.allocator.NewMemBatchWithMaxCapacity(a.outputTypes) } func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { @@ -191,10 +175,10 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { a.scratch.ResetInternalBatch() a.scratch.shouldResetInternalBatch = false } - if a.scratch.resumeIdx >= a.scratch.outputSize { + if a.scratch.resumeIdx >= coldata.BatchSize() { // Copy the second part of the output batch into the first and resume from // there. - newResumeIdx := a.scratch.resumeIdx - a.scratch.outputSize + newResumeIdx := a.scratch.resumeIdx - coldata.BatchSize() a.allocator.PerformOperation(a.scratch.ColVecs(), func() { for i := 0; i < len(a.outputTypes); i++ { vec := a.scratch.ColVec(i) @@ -208,12 +192,12 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { coldata.SliceArgs{ Src: vec, DestIdx: 0, - SrcStartIdx: a.scratch.outputSize, + SrcStartIdx: coldata.BatchSize(), SrcEndIdx: a.scratch.resumeIdx + 1, }, ) // Now we need to restore the desired length for the Vec. - vec.SetLength(a.scratch.inputSize) + vec.SetLength(2 * coldata.BatchSize()) a.aggregateFuncs[i].SetOutputIndex(newResumeIdx) // There might have been some NULLs set in the part that we // have just copied over, so we need to unset the NULLs. @@ -223,7 +207,7 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { a.scratch.resumeIdx = newResumeIdx } - for a.scratch.resumeIdx < a.scratch.outputSize { + for a.scratch.resumeIdx < coldata.BatchSize() { batch := a.input.Next(ctx) batchLength := batch.Length() a.seenNonEmptyBatch = a.seenNonEmptyBatch || batchLength > 0 @@ -270,8 +254,8 @@ func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { } batchToReturn := a.scratch.Batch - if a.scratch.resumeIdx > a.scratch.outputSize { - a.scratch.SetLength(a.scratch.outputSize) + if a.scratch.resumeIdx > coldata.BatchSize() { + a.scratch.SetLength(coldata.BatchSize()) a.allocator.PerformOperation(a.unsafeBatch.ColVecs(), func() { for i := 0; i < len(a.outputTypes); i++ { a.unsafeBatch.ColVec(i).Copy( diff --git a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go index 4a3f31f3193b..da65d419de15 100644 --- a/pkg/sql/colexec/parallel_unordered_synchronizer_test.go +++ b/pkg/sql/colexec/parallel_unordered_synchronizer_test.go @@ -182,7 +182,7 @@ func BenchmarkParallelUnorderedSynchronizer(b *testing.B) { typs := []*types.T{types.Int} inputs := make([]SynchronizerInput, numInputs) for i := range inputs { - batch := testAllocator.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()) + batch := testAllocator.NewMemBatchWithMaxCapacity(typs) batch.SetLength(coldata.BatchSize()) inputs[i].Op = colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs) } diff --git a/pkg/sql/colexec/relative_rank.eg.go b/pkg/sql/colexec/relative_rank.eg.go index 5e0fafc9ad02..777debcce98b 100644 --- a/pkg/sql/colexec/relative_rank.eg.go +++ b/pkg/sql/colexec/relative_rank.eg.go @@ -179,7 +179,7 @@ func (r *percentRankNoPartitionOp) Init() { r.bufferedTuples = newSpillingQueue( r.allocator, r.inputTypes, int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // All rank functions start counting from 1. Before we assign the rank to a @@ -373,13 +373,13 @@ func (r *percentRankWithPartitionOp) Init() { r.partitionsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction r.bufferedTuples = newSpillingQueue( r.allocator, r.inputTypes, int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // All rank functions start counting from 1. Before we assign the rank to a @@ -682,13 +682,13 @@ func (r *cumeDistNoPartitionOp) Init() { r.peerGroupsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction r.bufferedTuples = newSpillingQueue( r.allocator, r.inputTypes, int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) } @@ -979,19 +979,19 @@ func (r *cumeDistWithPartitionOp) Init() { r.partitionsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction r.peerGroupsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction r.bufferedTuples = newSpillingQueue( r.allocator, r.inputTypes, int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) } diff --git a/pkg/sql/colexec/relative_rank_tmpl.go b/pkg/sql/colexec/relative_rank_tmpl.go index 84bd57b293c9..dd26601612f3 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -283,7 +283,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { r.partitionsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction // {{end}} @@ -291,14 +291,14 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { r.peerGroupsState.spillingQueue = newSpillingQueue( r.allocator, []*types.T{types.Int}, int64(float64(r.memoryLimit)*relativeRankUtilityQueueMemLimitFraction), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) usedMemoryLimitFraction += relativeRankUtilityQueueMemLimitFraction // {{end}} r.bufferedTuples = newSpillingQueue( r.allocator, r.inputTypes, int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), - r.diskQueueCfg, r.fdSemaphore, coldata.BatchSize(), r.diskAcc, + r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) // {{if .IsPercentRank}} diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index 3c069cef14a0..561e18c307b9 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -177,9 +177,6 @@ type routerOutputOpTestingKnobs struct { // defaultRouterOutputBlockedThreshold but can be modified by tests to test // edge cases. blockedThreshold int - // outputBatchSize defaults to coldata.BatchSize() but can be modified by - // tests to test edge cases. - outputBatchSize int // alwaysFlush, if set to true, will always flush o.mu.pendingBatch to // o.mu.data. alwaysFlush bool @@ -224,9 +221,6 @@ func newRouterOutputOp(args routerOutputOpArgs) *routerOutputOp { if args.testingKnobs.blockedThreshold == 0 { args.testingKnobs.blockedThreshold = getDefaultRouterOutputBlockedThreshold() } - if args.testingKnobs.outputBatchSize == 0 { - args.testingKnobs.outputBatchSize = coldata.BatchSize() - } o := &routerOutputOp{ types: args.types, @@ -241,7 +235,6 @@ func newRouterOutputOp(args routerOutputOpArgs) *routerOutputOp { args.memoryLimit, args.cfg, args.fdSemaphore, - args.testingKnobs.outputBatchSize, args.diskAcc, ) @@ -419,9 +412,9 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele for toAppend := len(selection); toAppend > 0; { if o.mu.pendingBatch == nil { - o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithFixedCapacity(o.types, o.testingKnobs.outputBatchSize) + o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithMaxCapacity(o.types) } - available := o.testingKnobs.outputBatchSize - o.mu.pendingBatch.Length() + available := coldata.BatchSize() - o.mu.pendingBatch.Length() numAppended := toAppend if toAppend > available { numAppended = available @@ -442,7 +435,7 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele }) newLength := o.mu.pendingBatch.Length() + numAppended o.mu.pendingBatch.SetLength(newLength) - if o.testingKnobs.alwaysFlush || newLength >= o.testingKnobs.outputBatchSize { + if o.testingKnobs.alwaysFlush || newLength >= coldata.BatchSize() { // The capacity in o.mu.pendingBatch has been filled. err := o.mu.data.enqueue(ctx, o.mu.pendingBatch) if err == nil && o.testingKnobs.addBatchTestInducedErrorCb != nil { diff --git a/pkg/sql/colexec/routers_test.go b/pkg/sql/colexec/routers_test.go index 6ffed32e5ca1..d55a85d69bb3 100644 --- a/pkg/sql/colexec/routers_test.go +++ b/pkg/sql/colexec/routers_test.go @@ -165,7 +165,6 @@ func TestRouterOutputAddBatch(t *testing.T) { unblockedEventsChan: unblockEventsChan, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: tc.blockedThreshold, - outputBatchSize: tc.outputBatchSize, }, }, ) @@ -434,7 +433,6 @@ func TestRouterOutputRandom(t *testing.T) { var ( maxValues = coldata.BatchSize() * 4 blockedThreshold = 1 + rng.Intn(maxValues-1) - outputSize = 1 + rng.Intn(maxValues-1) ) typs := []*types.T{types.Int, types.Int} @@ -452,7 +450,7 @@ func TestRouterOutputRandom(t *testing.T) { defer cleanup() testName := fmt.Sprintf( - "blockedThreshold=%d/outputSize=%d/totalInputSize=%d", blockedThreshold, outputSize, len(data), + "blockedThreshold=%d/totalInputSize=%d", blockedThreshold, len(data), ) for _, mtc := range memoryTestCases { t.Run(fmt.Sprintf("%s/memoryLimit=%s", testName, humanizeutil.IBytes(mtc.bytes)), func(t *testing.T) { @@ -470,7 +468,6 @@ func TestRouterOutputRandom(t *testing.T) { unblockedEventsChan: unblockedEventsChans, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: blockedThreshold, - outputBatchSize: outputSize, }, }, ) @@ -858,7 +855,6 @@ func TestHashRouterRandom(t *testing.T) { maxValues = coldata.BatchSize() * 4 maxOutputs = 128 blockedThreshold = 1 + rng.Intn(maxValues-1) - outputSize = 1 + rng.Intn(maxValues-1) numOutputs = 1 + rng.Intn(maxOutputs-1) ) @@ -918,11 +914,10 @@ func TestHashRouterRandom(t *testing.T) { ) testName := fmt.Sprintf( - "terminationScenario=%d/numOutputs=%d/blockedThreshold=%d/outputSize=%d/totalInputSize=%d/hashCols=%v", + "terminationScenario=%d/numOutputs=%d/blockedThreshold=%d/totalInputSize=%d/hashCols=%v", terminationScenario, numOutputs, blockedThreshold, - outputSize, len(data), hashCols, ) @@ -961,7 +956,6 @@ func TestHashRouterRandom(t *testing.T) { unblockedEventsChan: unblockEventsChan, testingKnobs: routerOutputOpTestingKnobs{ blockedThreshold: blockedThreshold, - outputBatchSize: outputSize, }, }, ) diff --git a/pkg/sql/colexec/spilling_queue.go b/pkg/sql/colexec/spilling_queue.go index 7136e76501b2..a466e2a5f676 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -75,7 +75,6 @@ func newSpillingQueue( memoryLimit int64, cfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - batchSize int, diskAcc *mon.BoundAccount, ) *spillingQueue { // Reduce the memory limit by what the DiskQueue may need to buffer @@ -84,7 +83,7 @@ func newSpillingQueue( if memoryLimit < 0 { memoryLimit = 0 } - itemsLen := memoryLimit / int64(colmem.EstimateBatchSizeBytes(typs, batchSize)) + itemsLen := memoryLimit / int64(colmem.EstimateBatchSizeBytes(typs, coldata.BatchSize())) if itemsLen == 0 { // Make items at least of length 1. Even though batches will spill to disk // directly (this can only happen with a very low memory limit), it's nice @@ -114,10 +113,9 @@ func newRewindableSpillingQueue( memoryLimit int64, cfg colcontainer.DiskQueueCfg, fdSemaphore semaphore.Semaphore, - batchSize int, diskAcc *mon.BoundAccount, ) *spillingQueue { - q := newSpillingQueue(unlimitedAllocator, typs, memoryLimit, cfg, fdSemaphore, batchSize, diskAcc) + q := newSpillingQueue(unlimitedAllocator, typs, memoryLimit, cfg, fdSemaphore, diskAcc) q.rewindable = true return q } diff --git a/pkg/sql/colexec/spilling_queue_test.go b/pkg/sql/colexec/spilling_queue_test.go index ffd740a94ca4..e44ad3e6d047 100644 --- a/pkg/sql/colexec/spilling_queue_test.go +++ b/pkg/sql/colexec/spilling_queue_test.go @@ -79,14 +79,12 @@ func TestSpillingQueue(t *testing.T) { if rewindable { q = newRewindableSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(2), coldata.BatchSize(), - testDiskAcc, + colexecbase.NewTestingSemaphore(2), testDiskAcc, ) } else { q = newSpillingQueue( testAllocator, typs, memoryLimit, queueCfg, - colexecbase.NewTestingSemaphore(2), coldata.BatchSize(), - testDiskAcc, + colexecbase.NewTestingSemaphore(2), testDiskAcc, ) } diff --git a/pkg/sql/colexec/types_integration_test.go b/pkg/sql/colexec/types_integration_test.go index 9e0520444451..eb56464b440f 100644 --- a/pkg/sql/colexec/types_integration_test.go +++ b/pkg/sql/colexec/types_integration_test.go @@ -165,7 +165,7 @@ func (a *arrowTestOperator) Next(ctx context.Context) coldata.Batch { if err := a.r.Deserialize(&arrowDataOut, buf.Bytes()); err != nil { colexecerror.InternalError(err) } - batchOut := testAllocator.NewMemBatchWithFixedCapacity(a.typs, coldata.BatchSize()) + batchOut := testAllocator.NewMemBatchWithMaxCapacity(a.typs) if err := a.c.ArrowToBatch(arrowDataOut, batchOut); err != nil { colexecerror.InternalError(err) } diff --git a/pkg/sql/colexec/utils_test.go b/pkg/sql/colexec/utils_test.go index 2980cb63bad4..ff706ece4f9d 100644 --- a/pkg/sql/colexec/utils_test.go +++ b/pkg/sql/colexec/utils_test.go @@ -204,8 +204,6 @@ const ( unorderedVerifier ) -type verifierFn func(output *opTestOutput) error - // maybeHasNulls is a helper function that returns whether any of the columns in b // (maybe) have nulls. func maybeHasNulls(b coldata.Batch) bool { @@ -220,14 +218,7 @@ func maybeHasNulls(b coldata.Batch) bool { return false } -type testRunner func(*testing.T, []tuples, [][]*types.T, tuples, interface{}, func([]colexecbase.Operator) (colexecbase.Operator, error)) - -// variableOutputBatchSizeInitializer is implemented by operators that can be -// initialized with variable output size batches. This allows runTests to -// increase test coverage of these operators. -type variableOutputBatchSizeInitializer interface { - initWithOutputBatchSize(int) -} +type testRunner func(*testing.T, []tuples, [][]*types.T, tuples, verifierType, func([]colexecbase.Operator) (colexecbase.Operator, error)) // runTests is a helper that automatically runs your tests with varied batch // sizes and with and without a random selection vector. @@ -239,7 +230,7 @@ func runTests( t *testing.T, tups []tuples, expected tuples, - verifier interface{}, + verifier verifierType, constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithTyps(t, tups, nil /* typs */, expected, verifier, constructor) @@ -255,7 +246,7 @@ func runTestsWithTyps( tups []tuples, typs [][]*types.T, expected tuples, - verifier interface{}, + verifier verifierType, constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { runTestsWithoutAllNullsInjection(t, tups, typs, expected, verifier, constructor) @@ -351,28 +342,18 @@ func runTestsWithoutAllNullsInjection( tups []tuples, typs [][]*types.T, expected tuples, - verifier interface{}, + verifier verifierType, constructor func(inputs []colexecbase.Operator) (colexecbase.Operator, error), ) { ctx := context.Background() + verifyFn := (*opTestOutput).VerifyAnyOrder skipVerifySelAndNullsResets := true - var verifyFn verifierFn - switch v := verifier.(type) { - case verifierType: - switch v { - case orderedVerifier: - verifyFn = (*opTestOutput).Verify - // Note that this test makes sense only if we expect tuples to be - // returned in the same order (otherwise the second batch's selection - // vector or nulls info can be different and that is totally valid). - skipVerifySelAndNullsResets = false - case unorderedVerifier: - verifyFn = (*opTestOutput).VerifyAnyOrder - default: - colexecerror.InternalError(fmt.Sprintf("unexpected verifierType %d", v)) - } - case verifierFn: - verifyFn = v + if verifier == orderedVerifier { + verifyFn = (*opTestOutput).Verify + // Note that this test makes sense only if we expect tuples to be + // returned in the same order (otherwise the second batch's selection + // vector or nulls info can be different and that is totally valid). + skipVerifySelAndNullsResets = false } runTestsWithFn(t, tups, typs, func(t *testing.T, inputs []colexecbase.Operator) { op, err := constructor(inputs) @@ -413,13 +394,7 @@ func runTestsWithoutAllNullsInjection( if err != nil { t.Fatal(err) } - if vbsiOp, ok := op.(variableOutputBatchSizeInitializer); ok { - // initialize the operator with a very small output batch size to - // increase the likelihood that multiple batches will be output. - vbsiOp.initWithOutputBatchSize(1) - } else { - op.Init() - } + op.Init() b := op.Next(ctx) if b.Length() == 0 { return @@ -1470,16 +1445,11 @@ type joinTestCase struct { leftEqColsAreKey bool rightEqColsAreKey bool expected tuples - outputBatchSize int skipAllNullsInjection bool onExpr execinfrapb.Expression } func (tc *joinTestCase) init() { - if tc.outputBatchSize == 0 { - tc.outputBatchSize = coldata.BatchSize() - } - if len(tc.leftDirections) == 0 { tc.leftDirections = make([]execinfrapb.Ordering_Column_Direction, len(tc.leftTypes)) for i := range tc.leftDirections { From 895125bc90ce0d286eeabb49e4b7cf7a75322045 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 6 Aug 2020 17:46:50 -0700 Subject: [PATCH 3/3] colexec: use batches with dynamic capacity in several operators This commit introduces `ResetMaybeReallocate` method on `colmem.Allocator` which might allocate a new batch (it uses an exponential capacity growth until `coldata.BatchSize()` and also supports a minimum capacity argument). The method can be used by the operators that want "dynamic batch size" behavior. All usages of `NewMemBatchWithMaxCapacity` and `NewMemBatchWithFixedCapacity` in non-test files have been audited, and most of the operators have been updated to exhibit the dynamic batch size behavior (most notable exception to this are the aggregators because currently aggregate functions hold on their output vectors, so we can't just reallocate an output batch). The usage of `NewMemBatchWithMaxCapacity` is now prohibited in non-test files by a linter in order to encourage the engineers to think whether a dynamic batch size behavior is desired. Release note: None --- pkg/col/coldata/batch.go | 3 + pkg/sql/colexec/columnarizer.go | 22 +- pkg/sql/colexec/deselector.go | 18 +- pkg/sql/colexec/external_hash_joiner.go | 8 +- pkg/sql/colexec/hash_aggregator.go | 9 +- pkg/sql/colexec/hashjoiner.eg.go | 8 +- pkg/sql/colexec/hashjoiner.go | 82 ++-- pkg/sql/colexec/hashjoiner_tmpl.go | 4 +- pkg/sql/colexec/mergejoiner.go | 8 +- pkg/sql/colexec/mergejoiner_exceptall.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_fullouter.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_inner.eg.go | 374 +++++++++--------- .../colexec/mergejoiner_intersectall.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_leftanti.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_leftouter.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_leftsemi.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_rightouter.eg.go | 374 +++++++++--------- pkg/sql/colexec/mergejoiner_tmpl.go | 26 +- pkg/sql/colexec/ordered_aggregator.go | 7 +- pkg/sql/colexec/ordered_synchronizer.eg.go | 160 ++++---- pkg/sql/colexec/ordered_synchronizer_tmpl.go | 51 ++- pkg/sql/colexec/relative_rank.eg.go | 32 +- pkg/sql/colexec/relative_rank_tmpl.go | 10 +- pkg/sql/colexec/routers.go | 6 +- pkg/sql/colexec/sort.go | 95 +++-- pkg/sql/colexec/sorttopk.go | 50 +-- pkg/sql/colexec/spilling_queue.go | 2 +- pkg/sql/colexec/stats_test.go | 14 +- pkg/sql/colexec/unordered_distinct.go | 14 +- pkg/sql/colexec/utils.go | 8 +- pkg/sql/colfetcher/cfetcher.go | 50 ++- pkg/sql/colflow/colrpc/inbox.go | 8 +- pkg/sql/colmem/allocator.go | 63 ++- pkg/testutils/lint/lint_test.go | 36 ++ 34 files changed, 1969 insertions(+), 1817 deletions(-) diff --git a/pkg/col/coldata/batch.go b/pkg/col/coldata/batch.go index 57f58ae83937..a72a25c497e8 100644 --- a/pkg/col/coldata/batch.go +++ b/pkg/col/coldata/batch.go @@ -78,6 +78,9 @@ const defaultBatchSize = 1024 var batchSize int64 = defaultBatchSize // BatchSize is the maximum number of tuples that fit in a column batch. +// TODO(yuzefovich): we are treating this method almost as if it were a +// constant while it performs an atomic operation. Think through whether it has +// a noticeable performance hit. func BatchSize() int { return int(atomic.LoadInt64(&batchSize)) } diff --git a/pkg/sql/colexec/columnarizer.go b/pkg/sql/colexec/columnarizer.go index 17cae80a27b0..03151f93db27 100644 --- a/pkg/sql/colexec/columnarizer.go +++ b/pkg/sql/colexec/columnarizer.go @@ -81,11 +81,6 @@ func (c *Columnarizer) Init() { // internal objects several times if Init method is called more than once, so // we have this check in place. if c.initStatus == OperatorNotInitialized { - c.batch = c.allocator.NewMemBatchWithMaxCapacity(c.typs) - c.buffered = make(sqlbase.EncDatumRows, coldata.BatchSize()) - for i := range c.buffered { - c.buffered[i] = make(sqlbase.EncDatumRow, len(c.typs)) - } c.accumulatedMeta = make([]execinfrapb.ProducerMetadata, 0, 1) c.input.Start(c.ctx) c.initStatus = OperatorInitialized @@ -94,11 +89,24 @@ func (c *Columnarizer) Init() { // Next is part of the Operator interface. func (c *Columnarizer) Next(context.Context) coldata.Batch { - c.batch.ResetInternalBatch() + var reallocated bool + c.batch, reallocated = c.allocator.ResetMaybeReallocate(c.typs, c.batch, 1 /* minCapacity */) + if reallocated { + oldRows := c.buffered + c.buffered = make(sqlbase.EncDatumRows, c.batch.Capacity()) + for i := range c.buffered { + if len(oldRows) > 0 { + c.buffered[i] = oldRows[0] + oldRows = oldRows[1:] + } else { + c.buffered[i] = make(sqlbase.EncDatumRow, len(c.typs)) + } + } + } // Buffer up n rows. nRows := 0 columnTypes := c.OutputTypes() - for ; nRows < coldata.BatchSize(); nRows++ { + for ; nRows < c.batch.Capacity(); nRows++ { row, meta := c.input.Next() if meta != nil { nRows-- diff --git a/pkg/sql/colexec/deselector.go b/pkg/sql/colexec/deselector.go index bceecbb6d74d..90634bb7df3f 100644 --- a/pkg/sql/colexec/deselector.go +++ b/pkg/sql/colexec/deselector.go @@ -51,12 +51,18 @@ func (p *deselectorOp) Init() { } func (p *deselectorOp) Next(ctx context.Context) coldata.Batch { - p.resetOutput() + // TODO(yuzefovich): this allocation is only needed in order to appease the + // tests of the external sorter with forced disk spilling (if we don't do + // this, an OOM error occurs during ResetMaybeReallocate call below at + // which point we have already received a batch from the input and it'll + // get lost because deselectorOp doesn't support fall-over to the + // disk-backed infrastructure). + p.output, _ = p.allocator.ResetMaybeReallocate(p.inputTypes, p.output, 1 /* minCapacity */) batch := p.input.Next(ctx) if batch.Selection() == nil { return batch } - + p.output, _ = p.allocator.ResetMaybeReallocate(p.inputTypes, p.output, batch.Length()) sel := batch.Selection() p.allocator.PerformOperation(p.output.ColVecs(), func() { for i := range p.inputTypes { @@ -76,11 +82,3 @@ func (p *deselectorOp) Next(ctx context.Context) coldata.Batch { p.output.SetLength(batch.Length()) return p.output } - -func (p *deselectorOp) resetOutput() { - if p.output == nil { - p.output = p.allocator.NewMemBatchWithMaxCapacity(p.inputTypes) - } else { - p.output.ResetInternalBatch() - } -} diff --git a/pkg/sql/colexec/external_hash_joiner.go b/pkg/sql/colexec/external_hash_joiner.go index ca0969aa3024..e2843a4f4b1c 100644 --- a/pkg/sql/colexec/external_hash_joiner.go +++ b/pkg/sql/colexec/external_hash_joiner.go @@ -387,8 +387,8 @@ func NewExternalHashJoiner( if ehj.memState.maxRightPartitionSizeToJoin < externalHJMinimalMaxRightPartitionSize { ehj.memState.maxRightPartitionSizeToJoin = externalHJMinimalMaxRightPartitionSize } - ehj.scratch.leftBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.left.sourceTypes) - ehj.recursiveScratch.leftBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.left.sourceTypes) + ehj.scratch.leftBatch = unlimitedAllocator.NewMemBatchWithFixedCapacity(spec.left.sourceTypes, coldata.BatchSize()) + ehj.recursiveScratch.leftBatch = unlimitedAllocator.NewMemBatchWithFixedCapacity(spec.left.sourceTypes, coldata.BatchSize()) sameSourcesSchema := len(spec.left.sourceTypes) == len(spec.right.sourceTypes) for i, leftType := range spec.left.sourceTypes { if i < len(spec.right.sourceTypes) && !leftType.Identical(spec.right.sourceTypes[i]) { @@ -401,8 +401,8 @@ func NewExternalHashJoiner( ehj.scratch.rightBatch = ehj.scratch.leftBatch ehj.recursiveScratch.rightBatch = ehj.recursiveScratch.leftBatch } else { - ehj.scratch.rightBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.right.sourceTypes) - ehj.recursiveScratch.rightBatch = unlimitedAllocator.NewMemBatchWithMaxCapacity(spec.right.sourceTypes) + ehj.scratch.rightBatch = unlimitedAllocator.NewMemBatchWithFixedCapacity(spec.right.sourceTypes, coldata.BatchSize()) + ehj.recursiveScratch.rightBatch = unlimitedAllocator.NewMemBatchWithFixedCapacity(spec.right.sourceTypes, coldata.BatchSize()) } ehj.testingKnobs.numForcedRepartitions = numForcedRepartitions ehj.testingKnobs.delegateFDAcquisitions = delegateFDAcquisitions diff --git a/pkg/sql/colexec/hash_aggregator.go b/pkg/sql/colexec/hash_aggregator.go index d50a3529b362..84c7e3bc9fdd 100644 --- a/pkg/sql/colexec/hash_aggregator.go +++ b/pkg/sql/colexec/hash_aggregator.go @@ -142,7 +142,12 @@ func NewHashAggregator( func (op *hashAggregator) Init() { op.input.Init() - op.output = op.allocator.NewMemBatchWithMaxCapacity(op.outputTypes) + // Note that we use a batch with fixed capacity because aggregate functions + // hold onto the vectors passed in into their Init method, so we cannot + // simply reallocate the output batch. + // TODO(yuzefovich): consider changing aggregateFunc interface to allow for + // updating the output vector. + op.output = op.allocator.NewMemBatchWithFixedCapacity(op.outputTypes, coldata.BatchSize()) op.scratch.eqChains = make([][]int, coldata.BatchSize()) op.scratch.intSlice = make([]int, coldata.BatchSize()) op.scratch.anotherIntSlice = make([]int, coldata.BatchSize()) @@ -181,7 +186,7 @@ func (op *hashAggregator) Next(ctx context.Context) coldata.Batch { case hashAggregatorOutputting: op.output.ResetInternalBatch() curOutputIdx := 0 - for curOutputIdx < coldata.BatchSize() && curOutputIdx < len(op.buckets) { + for curOutputIdx < op.output.Capacity() && curOutputIdx < len(op.buckets) { bucket := op.buckets[curOutputIdx] for _, fn := range bucket.fns { fn.Flush(curOutputIdx) diff --git a/pkg/sql/colexec/hashjoiner.eg.go b/pkg/sql/colexec/hashjoiner.eg.go index 7c92c3fcf223..3c2ce9b6f9f8 100644 --- a/pkg/sql/colexec/hashjoiner.eg.go +++ b/pkg/sql/colexec/hashjoiner.eg.go @@ -24,7 +24,7 @@ func collectProbeOuter_false( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -70,7 +70,7 @@ func collectProbeOuter_true( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -116,7 +116,7 @@ func collectProbeNoOuter_false( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -148,7 +148,7 @@ func collectProbeNoOuter_true( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults diff --git a/pkg/sql/colexec/hashjoiner.go b/pkg/sql/colexec/hashjoiner.go index 7418a374309d..dfe336644b18 100644 --- a/pkg/sql/colexec/hashjoiner.go +++ b/pkg/sql/colexec/hashjoiner.go @@ -172,7 +172,8 @@ type hashJoiner struct { ht *hashTable // output stores the resulting output batch that is constructed and returned // for every input batch during the probe phase. - output coldata.Batch + output coldata.Batch + outputTypes []*types.T // probeState is used in hjProbing state. probeState struct { @@ -242,7 +243,6 @@ func (hj *hashJoiner) Init() { } func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { - hj.resetOutput() for { switch hj.state { case hjBuilding: @@ -256,14 +256,22 @@ func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { hj.spec.joinType == descpb.LeftSemiJoin || hj.spec.joinType == descpb.IntersectAllJoin { hj.state = hjDone - continue } + // The code that performs fall-over to disk-backed + // infrastructure (ExportBuffered) assumes that we haven't + // received any batches from the left side. We don't want to + // modify that assumption because it is non-trivial to track + // which tuples have been processed and which haven't. So in + // order to satisfy the assumption, when the right side is + // empty, we force an allocation of an output batch early (this + // is only necessary to appease "*-disk" logic tests + // configurations). + hj.resetOutput(0 /* nResults */) } continue case hjProbing: - hj.exec(ctx) - - if hj.output.Length() == 0 { + output := hj.exec(ctx) + if output.Length() == 0 { if hj.spec.right.outer { hj.state = hjEmittingUnmatched } else { @@ -271,7 +279,7 @@ func (hj *hashJoiner) Next(ctx context.Context) coldata.Batch { } continue } - return hj.output + return output case hjEmittingUnmatched: if hj.emittingUnmatchedState.rowIdx == hj.ht.vals.Length() { hj.state = hjDone @@ -313,21 +321,21 @@ func (hj *hashJoiner) build(ctx context.Context) { // that didn't get a match. This will be called only for RIGHT OUTER and FULL // OUTER joins. func (hj *hashJoiner) emitUnmatched() { - // Set all elements in the probe columns of the output batch to null. - for i := range hj.spec.left.sourceTypes { - outCol := hj.output.ColVec(i) - outCol.Nulls().SetNulls() - } - nResults := 0 - - for nResults < hj.output.Capacity() && hj.emittingUnmatchedState.rowIdx < hj.ht.vals.Length() { + for nResults < coldata.BatchSize() && hj.emittingUnmatchedState.rowIdx < hj.ht.vals.Length() { if !hj.probeState.buildRowMatched[hj.emittingUnmatchedState.rowIdx] { hj.probeState.buildIdx[nResults] = hj.emittingUnmatchedState.rowIdx nResults++ } hj.emittingUnmatchedState.rowIdx++ } + hj.resetOutput(nResults) + + // Set all elements in the probe columns of the output batch to null. + for i := range hj.spec.left.sourceTypes { + outCol := hj.output.ColVec(i) + outCol.Nulls().SetNullRange(0 /* startIdx */, nResults) + } outCols := hj.output.ColVecs()[len(hj.spec.left.sourceTypes) : len(hj.spec.left.sourceTypes)+len(hj.spec.right.sourceTypes)] for i := range hj.spec.right.sourceTypes { @@ -360,17 +368,8 @@ func (hj *hashJoiner) emitUnmatched() { // columns. It returns a Batch with N + M columns where N is the number of // left source columns and M is the number of right source columns. The first N // columns correspond to the respective left source columns, followed by the -// right source columns as the last M elements. Even though all the columns are -// present in the result, only the specified output columns store relevant -// information. The remaining columns are there as dummy columns and their -// states are undefined. -// -// rightDistinct is true if the build table equality columns are distinct. It -// performs the same operation as the exec() function normally would while -// taking a shortcut to improve speed. -func (hj *hashJoiner) exec(ctx context.Context) { - hj.output.SetLength(0) - +// right source columns as the last M elements. +func (hj *hashJoiner) exec(ctx context.Context) coldata.Batch { if batch := hj.probeState.prevBatch; batch != nil { // The previous result was bigger than the maximum batch size, so we didn't // finish outputting it in the last call to probe. Continue outputting the @@ -380,14 +379,14 @@ func (hj *hashJoiner) exec(ctx context.Context) { sel := batch.Selection() nResults := hj.collect(batch, batchSize, sel) - hj.congregate(nResults, batch, batchSize) + hj.congregate(nResults, batch) } else { for { batch := hj.inputOne.Next(ctx) batchSize := batch.Length() if batchSize == 0 { - break + return coldata.ZeroBatch } for i, colIdx := range hj.spec.left.eqCols { @@ -459,19 +458,20 @@ func (hj *hashJoiner) exec(ctx context.Context) { nResults = hj.collect(batch, batchSize, sel) } - hj.congregate(nResults, batch, batchSize) + hj.congregate(nResults, batch) if hj.output.Length() > 0 { break } } } + return hj.output } // congregate uses the probeIdx and buildIdx pairs to stitch together the // resulting join rows and add them to the output batch with the left table // columns preceding the right table columns. -func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize int) { +func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch) { // NOTE: Copy() calls are not accounted for because we don't want for memory // limit error to occur at this point - we have already built the hash // table and now are only consuming the left source one batch at a time, @@ -481,6 +481,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in // This behavior is acceptable - we allocated hj.output batch already, so the // concern here is only for the variable-sized types that exceed our // estimations. + hj.resetOutput(nResults) if hj.spec.joinType.ShouldIncludeRightColsInOutput() { rightColOffset := len(hj.spec.left.sourceTypes) @@ -510,7 +511,7 @@ func (hj *hashJoiner) congregate(nResults int, batch coldata.Batch, batchSize in for i := range hj.spec.right.sourceTypes { outCol := hj.output.ColVec(i + rightColOffset) nulls := outCol.Nulls() - for i, isNull := range hj.probeState.probeRowUnmatched { + for i, isNull := range hj.probeState.probeRowUnmatched[:nResults] { if isNull { nulls.SetNull(i) } @@ -587,16 +588,12 @@ func (hj *hashJoiner) ExportBuffered(input colexecbase.Operator) coldata.Batch { } } -func (hj *hashJoiner) resetOutput() { - if hj.output == nil { - outputTypes := append([]*types.T{}, hj.spec.left.sourceTypes...) - if hj.spec.joinType.ShouldIncludeRightColsInOutput() { - outputTypes = append(outputTypes, hj.spec.right.sourceTypes...) - } - hj.output = hj.allocator.NewMemBatchWithMaxCapacity(outputTypes) - } else { - hj.output.ResetInternalBatch() +func (hj *hashJoiner) resetOutput(nResults int) { + minCapacity := nResults + if minCapacity < 1 { + minCapacity = 1 } + hj.output, _ = hj.allocator.ResetMaybeReallocate(hj.outputTypes, hj.output, minCapacity) } func (hj *hashJoiner) reset(ctx context.Context) { @@ -684,10 +681,15 @@ func MakeHashJoinerSpec( func NewHashJoiner( allocator *colmem.Allocator, spec HashJoinerSpec, leftSource, rightSource colexecbase.Operator, ) colexecbase.Operator { + outputTypes := append([]*types.T{}, spec.left.sourceTypes...) + if spec.joinType.ShouldIncludeRightColsInOutput() { + outputTypes = append(outputTypes, spec.right.sourceTypes...) + } hj := &hashJoiner{ twoInputNode: newTwoInputNode(leftSource, rightSource), allocator: allocator, spec: spec, + outputTypes: outputTypes, } hj.probeState.buildIdx = make([]int, coldata.BatchSize()) hj.probeState.probeIdx = make([]int, coldata.BatchSize()) diff --git a/pkg/sql/colexec/hashjoiner_tmpl.go b/pkg/sql/colexec/hashjoiner_tmpl.go index 00a96abee4d2..9b4bacc54017 100644 --- a/pkg/sql/colexec/hashjoiner_tmpl.go +++ b/pkg/sql/colexec/hashjoiner_tmpl.go @@ -30,7 +30,7 @@ func collectProbeOuter( currentID := hj.ht.probeScratch.headID[i] for { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults @@ -71,7 +71,7 @@ func collectProbeNoOuter( for i := hj.probeState.prevBatchResumeIdx; i < batchSize; i++ { currentID := hj.ht.probeScratch.headID[i] for currentID != 0 { - if nResults >= hj.output.Capacity() { + if nResults >= coldata.BatchSize() { hj.probeState.prevBatch = batch hj.probeState.prevBatchResumeIdx = i return nResults diff --git a/pkg/sql/colexec/mergejoiner.go b/pkg/sql/colexec/mergejoiner.go index 57c26c8abad5..7378b7758fd5 100644 --- a/pkg/sql/colexec/mergejoiner.go +++ b/pkg/sql/colexec/mergejoiner.go @@ -488,7 +488,8 @@ type mergeJoinBase struct { right mergeJoinInput // Output buffer definition. - output coldata.Batch + output coldata.Batch + outputTypes []*types.T // outputReady is a flag to indicate that merge joiner is ready to emit an // output batch. outputReady bool @@ -530,11 +531,10 @@ func (o *mergeJoinBase) InternalMemoryUsage() int { } func (o *mergeJoinBase) Init() { - outputTypes := append([]*types.T{}, o.left.sourceTypes...) + o.outputTypes = append([]*types.T{}, o.left.sourceTypes...) if o.joinType.ShouldIncludeRightColsInOutput() { - outputTypes = append(outputTypes, o.right.sourceTypes...) + o.outputTypes = append(o.outputTypes, o.right.sourceTypes...) } - o.output = o.unlimitedAllocator.NewMemBatchWithMaxCapacity(outputTypes) o.left.source.Init() o.right.source.Init() o.proberState.lBufferedGroup.spillingQueue = newSpillingQueue( diff --git a/pkg/sql/colexec/mergejoiner_exceptall.eg.go b/pkg/sql/colexec/mergejoiner_exceptall.eg.go index 0952c250ae85..43548ec4c9e4 100644 --- a/pkg/sql/colexec/mergejoiner_exceptall.eg.go +++ b/pkg/sql/colexec/mergejoiner_exceptall.eg.go @@ -35559,8 +35559,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35624,8 +35624,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35689,8 +35689,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35753,8 +35753,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35814,8 +35814,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35876,8 +35876,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35941,8 +35941,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36006,8 +36006,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36071,8 +36071,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36136,8 +36136,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36207,8 +36207,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36269,8 +36269,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36331,8 +36331,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36392,8 +36392,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36450,8 +36450,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36509,8 +36509,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36571,8 +36571,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36633,8 +36633,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36695,8 +36695,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36757,8 +36757,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36826,8 +36826,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36890,8 +36890,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36954,8 +36954,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37017,8 +37017,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37077,8 +37077,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37138,8 +37138,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37202,8 +37202,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37266,8 +37266,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37330,8 +37330,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37394,8 +37394,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37464,8 +37464,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37525,8 +37525,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37586,8 +37586,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37646,8 +37646,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37703,8 +37703,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37761,8 +37761,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37822,8 +37822,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37883,8 +37883,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37944,8 +37944,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38005,8 +38005,8 @@ func (o *mergeJoinExceptAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38107,8 +38107,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38171,8 +38171,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38235,8 +38235,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38298,8 +38298,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38358,8 +38358,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38419,8 +38419,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38483,8 +38483,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38547,8 +38547,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38611,8 +38611,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38675,8 +38675,8 @@ func (o *mergeJoinExceptAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -38822,8 +38822,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38890,8 +38890,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38958,8 +38958,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39025,8 +39025,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39089,8 +39089,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39154,8 +39154,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39222,8 +39222,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39290,8 +39290,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39358,8 +39358,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39426,8 +39426,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39500,8 +39500,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39566,8 +39566,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39632,8 +39632,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39697,8 +39697,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39759,8 +39759,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39822,8 +39822,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39888,8 +39888,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39954,8 +39954,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40020,8 +40020,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40086,8 +40086,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40160,8 +40160,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40228,8 +40228,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40296,8 +40296,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40363,8 +40363,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40427,8 +40427,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40492,8 +40492,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40560,8 +40560,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40628,8 +40628,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40696,8 +40696,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40764,8 +40764,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40838,8 +40838,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40904,8 +40904,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -40970,8 +40970,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41035,8 +41035,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41097,8 +41097,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41160,8 +41160,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41226,8 +41226,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41292,8 +41292,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41358,8 +41358,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41424,8 +41424,8 @@ func (o *mergeJoinExceptAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -41518,8 +41518,8 @@ func (o *mergeJoinExceptAllOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -41944,9 +41944,9 @@ func (o *mergeJoinExceptAllOp) calculateOutputCount(groups []group) int { } count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -41976,7 +41976,7 @@ func (o *mergeJoinExceptAllOp) build(ctx context.Context) { } func (o *mergeJoinExceptAllOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -42031,7 +42031,7 @@ func (o *mergeJoinExceptAllOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_fullouter.eg.go b/pkg/sql/colexec/mergejoiner_fullouter.eg.go index 3b23d91904c1..9f7e7a40f6b7 100644 --- a/pkg/sql/colexec/mergejoiner_fullouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_fullouter.eg.go @@ -41740,8 +41740,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -41805,8 +41805,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -41870,8 +41870,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -41934,8 +41934,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -41995,8 +41995,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42057,8 +42057,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42122,8 +42122,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42187,8 +42187,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42252,8 +42252,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42317,8 +42317,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42388,8 +42388,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42450,8 +42450,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42512,8 +42512,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42573,8 +42573,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42631,8 +42631,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42690,8 +42690,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42752,8 +42752,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42814,8 +42814,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42876,8 +42876,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -42938,8 +42938,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43007,8 +43007,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43071,8 +43071,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43135,8 +43135,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43198,8 +43198,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43258,8 +43258,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43319,8 +43319,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43383,8 +43383,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43447,8 +43447,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43511,8 +43511,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43575,8 +43575,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43645,8 +43645,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43706,8 +43706,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43767,8 +43767,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43827,8 +43827,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43884,8 +43884,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -43942,8 +43942,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -44003,8 +44003,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -44064,8 +44064,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -44125,8 +44125,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -44186,8 +44186,8 @@ func (o *mergeJoinFullOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -44290,8 +44290,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44339,8 +44339,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44388,8 +44388,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44436,8 +44436,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44481,8 +44481,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44527,8 +44527,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44576,8 +44576,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44625,8 +44625,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44674,8 +44674,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44723,8 +44723,8 @@ func (o *mergeJoinFullOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -44856,8 +44856,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -44926,8 +44926,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -44996,8 +44996,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45065,8 +45065,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45131,8 +45131,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45198,8 +45198,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45268,8 +45268,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45338,8 +45338,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45408,8 +45408,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45478,8 +45478,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45554,8 +45554,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45622,8 +45622,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45690,8 +45690,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45757,8 +45757,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45821,8 +45821,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45886,8 +45886,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -45954,8 +45954,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46022,8 +46022,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46090,8 +46090,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46158,8 +46158,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46234,8 +46234,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46304,8 +46304,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46374,8 +46374,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46443,8 +46443,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46509,8 +46509,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46576,8 +46576,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46646,8 +46646,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46716,8 +46716,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46786,8 +46786,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46856,8 +46856,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -46932,8 +46932,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47000,8 +47000,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47068,8 +47068,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47135,8 +47135,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47199,8 +47199,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47264,8 +47264,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47332,8 +47332,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47400,8 +47400,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47468,8 +47468,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47536,8 +47536,8 @@ func (o *mergeJoinFullOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -47632,8 +47632,8 @@ func (o *mergeJoinFullOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -48077,9 +48077,9 @@ func (o *mergeJoinFullOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -48111,7 +48111,7 @@ func (o *mergeJoinFullOuterOp) build(ctx context.Context) { } func (o *mergeJoinFullOuterOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -48177,7 +48177,7 @@ func (o *mergeJoinFullOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_inner.eg.go b/pkg/sql/colexec/mergejoiner_inner.eg.go index e8009e037a06..54df5f9f0827 100644 --- a/pkg/sql/colexec/mergejoiner_inner.eg.go +++ b/pkg/sql/colexec/mergejoiner_inner.eg.go @@ -26092,8 +26092,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26154,8 +26154,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26216,8 +26216,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26277,8 +26277,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26335,8 +26335,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26394,8 +26394,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26456,8 +26456,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26518,8 +26518,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26580,8 +26580,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26642,8 +26642,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26710,8 +26710,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26769,8 +26769,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26828,8 +26828,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26886,8 +26886,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26941,8 +26941,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26997,8 +26997,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27056,8 +27056,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27115,8 +27115,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27174,8 +27174,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27233,8 +27233,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27299,8 +27299,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27360,8 +27360,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27421,8 +27421,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27481,8 +27481,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27538,8 +27538,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27596,8 +27596,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27657,8 +27657,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27718,8 +27718,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27779,8 +27779,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27840,8 +27840,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27907,8 +27907,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27965,8 +27965,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28023,8 +28023,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28080,8 +28080,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28134,8 +28134,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28189,8 +28189,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28247,8 +28247,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28305,8 +28305,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28363,8 +28363,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28421,8 +28421,8 @@ func (o *mergeJoinInnerOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28522,8 +28522,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28571,8 +28571,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28620,8 +28620,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28668,8 +28668,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28713,8 +28713,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28759,8 +28759,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28808,8 +28808,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28857,8 +28857,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28906,8 +28906,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28955,8 +28955,8 @@ func (o *mergeJoinInnerOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -29088,8 +29088,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29156,8 +29156,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29224,8 +29224,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29291,8 +29291,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29355,8 +29355,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29420,8 +29420,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29488,8 +29488,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29556,8 +29556,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29624,8 +29624,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29692,8 +29692,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29766,8 +29766,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29832,8 +29832,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29898,8 +29898,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29963,8 +29963,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30025,8 +30025,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30088,8 +30088,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30154,8 +30154,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30220,8 +30220,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30286,8 +30286,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30352,8 +30352,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30426,8 +30426,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30494,8 +30494,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30562,8 +30562,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30629,8 +30629,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30693,8 +30693,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30758,8 +30758,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30826,8 +30826,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30894,8 +30894,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30962,8 +30962,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31030,8 +31030,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31104,8 +31104,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31170,8 +31170,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31236,8 +31236,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31301,8 +31301,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31363,8 +31363,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31426,8 +31426,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31492,8 +31492,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31558,8 +31558,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31624,8 +31624,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31690,8 +31690,8 @@ func (o *mergeJoinInnerOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31784,8 +31784,8 @@ func (o *mergeJoinInnerOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -32191,9 +32191,9 @@ func (o *mergeJoinInnerOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -32225,7 +32225,7 @@ func (o *mergeJoinInnerOp) build(ctx context.Context) { } func (o *mergeJoinInnerOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -32264,7 +32264,7 @@ func (o *mergeJoinInnerOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_intersectall.eg.go b/pkg/sql/colexec/mergejoiner_intersectall.eg.go index 140fe02407f0..d55a8685813b 100644 --- a/pkg/sql/colexec/mergejoiner_intersectall.eg.go +++ b/pkg/sql/colexec/mergejoiner_intersectall.eg.go @@ -27092,8 +27092,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27154,8 +27154,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27216,8 +27216,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27277,8 +27277,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27335,8 +27335,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27394,8 +27394,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27456,8 +27456,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27518,8 +27518,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27580,8 +27580,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27642,8 +27642,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27710,8 +27710,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27769,8 +27769,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27828,8 +27828,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27886,8 +27886,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27941,8 +27941,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27997,8 +27997,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28056,8 +28056,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28115,8 +28115,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28174,8 +28174,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28233,8 +28233,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28299,8 +28299,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28360,8 +28360,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28421,8 +28421,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28481,8 +28481,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28538,8 +28538,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28596,8 +28596,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28657,8 +28657,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28718,8 +28718,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28779,8 +28779,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28840,8 +28840,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28907,8 +28907,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28965,8 +28965,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29023,8 +29023,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29080,8 +29080,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29134,8 +29134,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29189,8 +29189,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29247,8 +29247,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29305,8 +29305,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29363,8 +29363,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29421,8 +29421,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29523,8 +29523,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29587,8 +29587,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29651,8 +29651,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29714,8 +29714,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29774,8 +29774,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29835,8 +29835,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29899,8 +29899,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -29963,8 +29963,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30027,8 +30027,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30091,8 +30091,8 @@ func (o *mergeJoinIntersectAllOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if o.builderState.left.setOpLeftSrcIdx == leftGroup.rowEndIdx { @@ -30238,8 +30238,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30306,8 +30306,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30374,8 +30374,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30441,8 +30441,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30505,8 +30505,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30570,8 +30570,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30638,8 +30638,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30706,8 +30706,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30774,8 +30774,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30842,8 +30842,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30916,8 +30916,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30982,8 +30982,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31048,8 +31048,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31113,8 +31113,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31175,8 +31175,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31238,8 +31238,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31304,8 +31304,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31370,8 +31370,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31436,8 +31436,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31502,8 +31502,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31576,8 +31576,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31644,8 +31644,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31712,8 +31712,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31779,8 +31779,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31843,8 +31843,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31908,8 +31908,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31976,8 +31976,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32044,8 +32044,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32112,8 +32112,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32180,8 +32180,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32254,8 +32254,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32320,8 +32320,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32386,8 +32386,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32451,8 +32451,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32513,8 +32513,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32576,8 +32576,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32642,8 +32642,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32708,8 +32708,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32774,8 +32774,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32840,8 +32840,8 @@ func (o *mergeJoinIntersectAllOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -32934,8 +32934,8 @@ func (o *mergeJoinIntersectAllOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -33341,9 +33341,9 @@ func (o *mergeJoinIntersectAllOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -33373,7 +33373,7 @@ func (o *mergeJoinIntersectAllOp) build(ctx context.Context) { } func (o *mergeJoinIntersectAllOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -33412,7 +33412,7 @@ func (o *mergeJoinIntersectAllOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftanti.eg.go b/pkg/sql/colexec/mergejoiner_leftanti.eg.go index b8c89197ec8e..2cb106a8378b 100644 --- a/pkg/sql/colexec/mergejoiner_leftanti.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftanti.eg.go @@ -33919,8 +33919,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -33984,8 +33984,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34049,8 +34049,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34113,8 +34113,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34174,8 +34174,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34236,8 +34236,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34301,8 +34301,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34366,8 +34366,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34431,8 +34431,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34496,8 +34496,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34567,8 +34567,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34629,8 +34629,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34691,8 +34691,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34752,8 +34752,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34810,8 +34810,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34869,8 +34869,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34931,8 +34931,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34993,8 +34993,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35055,8 +35055,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35117,8 +35117,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35186,8 +35186,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35250,8 +35250,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35314,8 +35314,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35377,8 +35377,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35437,8 +35437,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35498,8 +35498,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35562,8 +35562,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35626,8 +35626,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35690,8 +35690,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35754,8 +35754,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35824,8 +35824,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35885,8 +35885,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35946,8 +35946,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36006,8 +36006,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36063,8 +36063,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36121,8 +36121,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36182,8 +36182,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36243,8 +36243,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36304,8 +36304,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36365,8 +36365,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36466,8 +36466,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36515,8 +36515,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36564,8 +36564,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36612,8 +36612,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36657,8 +36657,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36703,8 +36703,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36752,8 +36752,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36801,8 +36801,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36850,8 +36850,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36899,8 +36899,8 @@ func (o *mergeJoinLeftAntiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -37032,8 +37032,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37100,8 +37100,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37168,8 +37168,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37235,8 +37235,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37299,8 +37299,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37364,8 +37364,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37432,8 +37432,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37500,8 +37500,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37568,8 +37568,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37636,8 +37636,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37710,8 +37710,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37776,8 +37776,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37842,8 +37842,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37907,8 +37907,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37969,8 +37969,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38032,8 +38032,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38098,8 +38098,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38164,8 +38164,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38230,8 +38230,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38296,8 +38296,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38370,8 +38370,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38438,8 +38438,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38506,8 +38506,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38573,8 +38573,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38637,8 +38637,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38702,8 +38702,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38770,8 +38770,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38838,8 +38838,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38906,8 +38906,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38974,8 +38974,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39048,8 +39048,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39114,8 +39114,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39180,8 +39180,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39245,8 +39245,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39307,8 +39307,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39370,8 +39370,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39436,8 +39436,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39502,8 +39502,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39568,8 +39568,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39634,8 +39634,8 @@ func (o *mergeJoinLeftAntiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39728,8 +39728,8 @@ func (o *mergeJoinLeftAntiOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -40138,9 +40138,9 @@ func (o *mergeJoinLeftAntiOp) calculateOutputCount(groups []group) int { } count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -40170,7 +40170,7 @@ func (o *mergeJoinLeftAntiOp) build(ctx context.Context) { } func (o *mergeJoinLeftAntiOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -40225,7 +40225,7 @@ func (o *mergeJoinLeftAntiOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftouter.eg.go b/pkg/sql/colexec/mergejoiner_leftouter.eg.go index 108d595ee4d8..e0c74eef377c 100644 --- a/pkg/sql/colexec/mergejoiner_leftouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftouter.eg.go @@ -34076,8 +34076,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34138,8 +34138,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34200,8 +34200,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34261,8 +34261,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34319,8 +34319,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34378,8 +34378,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34440,8 +34440,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34502,8 +34502,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34564,8 +34564,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34626,8 +34626,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34694,8 +34694,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34753,8 +34753,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34812,8 +34812,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34870,8 +34870,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34925,8 +34925,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -34981,8 +34981,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35040,8 +35040,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35099,8 +35099,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35158,8 +35158,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35217,8 +35217,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35283,8 +35283,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35344,8 +35344,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35405,8 +35405,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35465,8 +35465,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35522,8 +35522,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35580,8 +35580,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35641,8 +35641,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35702,8 +35702,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35763,8 +35763,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35824,8 +35824,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35891,8 +35891,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -35949,8 +35949,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36007,8 +36007,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36064,8 +36064,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36118,8 +36118,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36173,8 +36173,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36231,8 +36231,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36289,8 +36289,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36347,8 +36347,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36405,8 +36405,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36506,8 +36506,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36555,8 +36555,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36604,8 +36604,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36652,8 +36652,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36697,8 +36697,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36743,8 +36743,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36792,8 +36792,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36841,8 +36841,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36890,8 +36890,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36939,8 +36939,8 @@ func (o *mergeJoinLeftOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -37072,8 +37072,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37142,8 +37142,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37212,8 +37212,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37281,8 +37281,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37347,8 +37347,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37414,8 +37414,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37484,8 +37484,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37554,8 +37554,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37624,8 +37624,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37694,8 +37694,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37770,8 +37770,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37838,8 +37838,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37906,8 +37906,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -37973,8 +37973,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38037,8 +38037,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38102,8 +38102,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38170,8 +38170,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38238,8 +38238,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38306,8 +38306,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38374,8 +38374,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38450,8 +38450,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38520,8 +38520,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38590,8 +38590,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38659,8 +38659,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38725,8 +38725,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38792,8 +38792,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38862,8 +38862,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -38932,8 +38932,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39002,8 +39002,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39072,8 +39072,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39148,8 +39148,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39216,8 +39216,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39284,8 +39284,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39351,8 +39351,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39415,8 +39415,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39480,8 +39480,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39548,8 +39548,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39616,8 +39616,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39684,8 +39684,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39752,8 +39752,8 @@ func (o *mergeJoinLeftOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if rightGroup.nullGroup { @@ -39848,8 +39848,8 @@ func (o *mergeJoinLeftOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -40275,9 +40275,9 @@ func (o *mergeJoinLeftOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -40309,7 +40309,7 @@ func (o *mergeJoinLeftOuterOp) build(ctx context.Context) { } func (o *mergeJoinLeftOuterOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -40364,7 +40364,7 @@ func (o *mergeJoinLeftOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go index 5fc484f408cb..cb06770f1f76 100644 --- a/pkg/sql/colexec/mergejoiner_leftsemi.eg.go +++ b/pkg/sql/colexec/mergejoiner_leftsemi.eg.go @@ -25932,8 +25932,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -25994,8 +25994,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26056,8 +26056,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26117,8 +26117,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26175,8 +26175,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26234,8 +26234,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26296,8 +26296,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26358,8 +26358,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26420,8 +26420,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26482,8 +26482,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26550,8 +26550,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26609,8 +26609,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26668,8 +26668,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26726,8 +26726,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26781,8 +26781,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26837,8 +26837,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26896,8 +26896,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -26955,8 +26955,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27014,8 +27014,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27073,8 +27073,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27139,8 +27139,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27200,8 +27200,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27261,8 +27261,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27321,8 +27321,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27378,8 +27378,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27436,8 +27436,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27497,8 +27497,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27558,8 +27558,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27619,8 +27619,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27680,8 +27680,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27747,8 +27747,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27805,8 +27805,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27863,8 +27863,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27920,8 +27920,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -27974,8 +27974,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28029,8 +28029,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28087,8 +28087,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28145,8 +28145,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28203,8 +28203,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28261,8 +28261,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28362,8 +28362,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28411,8 +28411,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28460,8 +28460,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28508,8 +28508,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28553,8 +28553,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28599,8 +28599,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28648,8 +28648,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28697,8 +28697,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28746,8 +28746,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28795,8 +28795,8 @@ func (o *mergeJoinLeftSemiOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -28928,8 +28928,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -28996,8 +28996,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29064,8 +29064,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29131,8 +29131,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29195,8 +29195,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29260,8 +29260,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29328,8 +29328,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29396,8 +29396,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29464,8 +29464,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29532,8 +29532,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29606,8 +29606,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29672,8 +29672,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29738,8 +29738,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29803,8 +29803,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29865,8 +29865,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29928,8 +29928,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -29994,8 +29994,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30060,8 +30060,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30126,8 +30126,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30192,8 +30192,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30266,8 +30266,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30334,8 +30334,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30402,8 +30402,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30469,8 +30469,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30533,8 +30533,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30598,8 +30598,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30666,8 +30666,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30734,8 +30734,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30802,8 +30802,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30870,8 +30870,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -30944,8 +30944,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31010,8 +31010,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31076,8 +31076,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31141,8 +31141,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31203,8 +31203,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31266,8 +31266,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31332,8 +31332,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31398,8 +31398,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31464,8 +31464,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31530,8 +31530,8 @@ func (o *mergeJoinLeftSemiOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -31624,8 +31624,8 @@ func (o *mergeJoinLeftSemiOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -32028,9 +32028,9 @@ func (o *mergeJoinLeftSemiOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -32060,7 +32060,7 @@ func (o *mergeJoinLeftSemiOp) build(ctx context.Context) { } func (o *mergeJoinLeftSemiOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -32099,7 +32099,7 @@ func (o *mergeJoinLeftSemiOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_rightouter.eg.go b/pkg/sql/colexec/mergejoiner_rightouter.eg.go index 523ddf4934a5..47fe21825961 100644 --- a/pkg/sql/colexec/mergejoiner_rightouter.eg.go +++ b/pkg/sql/colexec/mergejoiner_rightouter.eg.go @@ -33756,8 +33756,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -33821,8 +33821,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -33886,8 +33886,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -33950,8 +33950,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34011,8 +34011,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34073,8 +34073,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34138,8 +34138,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34203,8 +34203,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34268,8 +34268,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34333,8 +34333,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34404,8 +34404,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34466,8 +34466,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34528,8 +34528,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34589,8 +34589,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34647,8 +34647,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34706,8 +34706,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34768,8 +34768,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34830,8 +34830,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34892,8 +34892,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -34954,8 +34954,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35023,8 +35023,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35087,8 +35087,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35151,8 +35151,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35214,8 +35214,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35274,8 +35274,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35335,8 +35335,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35399,8 +35399,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35463,8 +35463,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35527,8 +35527,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35591,8 +35591,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35661,8 +35661,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35722,8 +35722,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35783,8 +35783,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35843,8 +35843,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35900,8 +35900,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -35958,8 +35958,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -36019,8 +36019,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -36080,8 +36080,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -36141,8 +36141,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -36202,8 +36202,8 @@ func (o *mergeJoinRightOuterOp) buildLeftGroupsFromBatch( repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if leftGroup.nullGroup { @@ -36306,8 +36306,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36355,8 +36355,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36404,8 +36404,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36452,8 +36452,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36497,8 +36497,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36543,8 +36543,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36592,8 +36592,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36641,8 +36641,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36690,8 +36690,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36739,8 +36739,8 @@ func (o *mergeJoinRightOuterOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } if src.Nulls().NullAt(srcStartIdx) { @@ -36872,8 +36872,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -36940,8 +36940,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37008,8 +37008,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37075,8 +37075,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37139,8 +37139,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37204,8 +37204,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37272,8 +37272,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37340,8 +37340,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37408,8 +37408,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37476,8 +37476,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37550,8 +37550,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37616,8 +37616,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37682,8 +37682,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37747,8 +37747,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37809,8 +37809,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37872,8 +37872,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -37938,8 +37938,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38004,8 +38004,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38070,8 +38070,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38136,8 +38136,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38210,8 +38210,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38278,8 +38278,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38346,8 +38346,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38413,8 +38413,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38477,8 +38477,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38542,8 +38542,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38610,8 +38610,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38678,8 +38678,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38746,8 +38746,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38814,8 +38814,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38888,8 +38888,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -38954,8 +38954,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39020,8 +39020,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39085,8 +39085,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39147,8 +39147,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39210,8 +39210,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39276,8 +39276,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39342,8 +39342,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39408,8 +39408,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39474,8 +39474,8 @@ func (o *mergeJoinRightOuterOp) buildRightGroupsFromBatch( o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } { @@ -39568,8 +39568,8 @@ func (o *mergeJoinRightOuterOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -39993,9 +39993,9 @@ func (o *mergeJoinRightOuterOp) calculateOutputCount(groups []group) int { for i := 0; i < len(groups); i++ { count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -40027,7 +40027,7 @@ func (o *mergeJoinRightOuterOp) build(ctx context.Context) { } func (o *mergeJoinRightOuterOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -40082,7 +40082,7 @@ func (o *mergeJoinRightOuterOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/mergejoiner_tmpl.go b/pkg/sql/colexec/mergejoiner_tmpl.go index 2662a9732d33..d880c1089fc9 100644 --- a/pkg/sql/colexec/mergejoiner_tmpl.go +++ b/pkg/sql/colexec/mergejoiner_tmpl.go @@ -747,8 +747,8 @@ func _LEFT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool) repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // {{if _JOIN_TYPE.IsRightOuter}} @@ -934,8 +934,8 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildLeftBufferedGroup( srcStartIdx := o.builderState.left.curSrcStartIdx repeatsLeft := leftGroup.numRepeats - o.builderState.left.numRepeatsIdx toAppend := repeatsLeft - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // {{if _JOIN_TYPE.IsSetOp}} @@ -1064,8 +1064,8 @@ func _RIGHT_SWITCH(_JOIN_TYPE joinTypeInfo, _HAS_SELECTION bool, _HAS_NULLS bool o.builderState.right.curSrcStartIdx = rightGroup.rowStartIdx } toAppend := rightGroup.rowEndIdx - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // {{if _JOIN_TYPE.IsLeftOuter}} @@ -1241,8 +1241,8 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) buildRightBufferedGroup( batchLength := currentBatch.Length() for batchLength > 0 { toAppend := batchLength - o.builderState.right.curSrcStartIdx - if outStartIdx+toAppend > coldata.BatchSize() { - toAppend = coldata.BatchSize() - outStartIdx + if outStartIdx+toAppend > o.output.Capacity() { + toAppend = o.output.Capacity() - outStartIdx } // Loop over every column. @@ -1506,9 +1506,9 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) calculateOutputCount(groups []group) int // {{end}} count += groups[i].toBuild groups[i].toBuild = 0 - if count > coldata.BatchSize() { - groups[i].toBuild = count - coldata.BatchSize() - count = coldata.BatchSize() + if count > o.output.Capacity() { + groups[i].toBuild = count - o.output.Capacity() + count = o.output.Capacity() return count } } @@ -1594,7 +1594,7 @@ func _SOURCE_FINISHED_SWITCH(_JOIN_TYPE joinTypeInfo) { // */}} // */}} func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { - o.output.ResetInternalBatch() + o.output, _ = o.unlimitedAllocator.ResetMaybeReallocate(o.outputTypes, o.output, 1 /* minCapacity */) for { switch o.state { case mjEntry: @@ -1630,7 +1630,7 @@ func (o *mergeJoin_JOIN_TYPE_STRINGOp) Next(ctx context.Context) coldata.Batch { o.builderState.outFinished = false } - if o.outputReady || o.builderState.outCount == coldata.BatchSize() { + if o.outputReady || o.builderState.outCount == o.output.Capacity() { if o.builderState.outCount == 0 { // We have already fully emitted the result of the join, so we // transition to "finished" state. diff --git a/pkg/sql/colexec/ordered_aggregator.go b/pkg/sql/colexec/ordered_aggregator.go index d4de2ca0cb9e..1a0d627509fb 100644 --- a/pkg/sql/colexec/ordered_aggregator.go +++ b/pkg/sql/colexec/ordered_aggregator.go @@ -163,7 +163,12 @@ func (a *orderedAggregator) Init() { vec := a.scratch.ColVec(i) a.aggregateFuncs[i].Init(a.groupCol, vec) } - a.unsafeBatch = a.allocator.NewMemBatchWithMaxCapacity(a.outputTypes) + // Note that we use a batch with fixed capacity because aggregate functions + // hold onto the vectors passed in into their Init method, so we cannot + // simply reallocate the output batch. + // TODO(yuzefovich): consider changing aggregateFunc interface to allow for + // updating the output vector. + a.unsafeBatch = a.allocator.NewMemBatchWithFixedCapacity(a.outputTypes, coldata.BatchSize()) } func (a *orderedAggregator) Next(ctx context.Context) coldata.Batch { diff --git a/pkg/sql/colexec/ordered_synchronizer.eg.go b/pkg/sql/colexec/ordered_synchronizer.eg.go index 9f4ff1b63cdd..40dde29dba14 100644 --- a/pkg/sql/colexec/ordered_synchronizer.eg.go +++ b/pkg/sql/colexec/ordered_synchronizer.eg.go @@ -121,10 +121,10 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { } heap.Init(o) } - o.output.ResetInternalBatch() + o.resetOutput() outputIdx := 0 o.allocator.PerformOperation(o.output.ColVecs(), func() { - for outputIdx < coldata.BatchSize() { + for outputIdx < o.output.Capacity() { if o.Len() == 0 { // All inputs exhausted. break @@ -253,81 +253,97 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { return o.output } +func (o *OrderedSynchronizer) resetOutput() { + var reallocated bool + o.output, reallocated = o.allocator.ResetMaybeReallocate(o.typs, o.output, 1 /* minCapacity */) + if reallocated { + o.outBoolCols = o.outBoolCols[:0] + o.outBytesCols = o.outBytesCols[:0] + o.outDecimalCols = o.outDecimalCols[:0] + o.outInt16Cols = o.outInt16Cols[:0] + o.outInt32Cols = o.outInt32Cols[:0] + o.outInt64Cols = o.outInt64Cols[:0] + o.outFloat64Cols = o.outFloat64Cols[:0] + o.outTimestampCols = o.outTimestampCols[:0] + o.outIntervalCols = o.outIntervalCols[:0] + o.outDatumCols = o.outDatumCols[:0] + for i, outVec := range o.output.ColVecs() { + o.outNulls[i] = outVec.Nulls() + switch typeconv.TypeFamilyToCanonicalTypeFamily(o.typs[i].Family()) { + case types.BoolFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outBoolCols) + o.outBoolCols = append(o.outBoolCols, outVec.Bool()) + } + case types.BytesFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outBytesCols) + o.outBytesCols = append(o.outBytesCols, outVec.Bytes()) + } + case types.DecimalFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outDecimalCols) + o.outDecimalCols = append(o.outDecimalCols, outVec.Decimal()) + } + case types.IntFamily: + switch o.typs[i].Width() { + case 16: + o.outColsMap[i] = len(o.outInt16Cols) + o.outInt16Cols = append(o.outInt16Cols, outVec.Int16()) + case 32: + o.outColsMap[i] = len(o.outInt32Cols) + o.outInt32Cols = append(o.outInt32Cols, outVec.Int32()) + case -1: + default: + o.outColsMap[i] = len(o.outInt64Cols) + o.outInt64Cols = append(o.outInt64Cols, outVec.Int64()) + } + case types.FloatFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outFloat64Cols) + o.outFloat64Cols = append(o.outFloat64Cols, outVec.Float64()) + } + case types.TimestampTZFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outTimestampCols) + o.outTimestampCols = append(o.outTimestampCols, outVec.Timestamp()) + } + case types.IntervalFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outIntervalCols) + o.outIntervalCols = append(o.outIntervalCols, outVec.Interval()) + } + case typeconv.DatumVecCanonicalTypeFamily: + switch o.typs[i].Width() { + case -1: + default: + o.outColsMap[i] = len(o.outDatumCols) + o.outDatumCols = append(o.outDatumCols, outVec.Datum()) + } + default: + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) + } + } + } +} + // Init is part of the Operator interface. func (o *OrderedSynchronizer) Init() { o.inputIndices = make([]int, len(o.inputs)) - o.output = o.allocator.NewMemBatchWithMaxCapacity(o.typs) o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) - for i, outVec := range o.output.ColVecs() { - o.outNulls[i] = outVec.Nulls() - switch typeconv.TypeFamilyToCanonicalTypeFamily(o.typs[i].Family()) { - case types.BoolFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outBoolCols) - o.outBoolCols = append(o.outBoolCols, outVec.Bool()) - } - case types.BytesFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outBytesCols) - o.outBytesCols = append(o.outBytesCols, outVec.Bytes()) - } - case types.DecimalFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outDecimalCols) - o.outDecimalCols = append(o.outDecimalCols, outVec.Decimal()) - } - case types.IntFamily: - switch o.typs[i].Width() { - case 16: - o.outColsMap[i] = len(o.outInt16Cols) - o.outInt16Cols = append(o.outInt16Cols, outVec.Int16()) - case 32: - o.outColsMap[i] = len(o.outInt32Cols) - o.outInt32Cols = append(o.outInt32Cols, outVec.Int32()) - case -1: - default: - o.outColsMap[i] = len(o.outInt64Cols) - o.outInt64Cols = append(o.outInt64Cols, outVec.Int64()) - } - case types.FloatFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outFloat64Cols) - o.outFloat64Cols = append(o.outFloat64Cols, outVec.Float64()) - } - case types.TimestampTZFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outTimestampCols) - o.outTimestampCols = append(o.outTimestampCols, outVec.Timestamp()) - } - case types.IntervalFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outIntervalCols) - o.outIntervalCols = append(o.outIntervalCols, outVec.Interval()) - } - case typeconv.DatumVecCanonicalTypeFamily: - switch o.typs[i].Width() { - case -1: - default: - o.outColsMap[i] = len(o.outDatumCols) - o.outDatumCols = append(o.outDatumCols, outVec.Datum()) - } - default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) - } - } for i := range o.inputs { o.inputs[i].Op.Init() } diff --git a/pkg/sql/colexec/ordered_synchronizer_tmpl.go b/pkg/sql/colexec/ordered_synchronizer_tmpl.go index 6907e71697ae..f904dbaec65f 100644 --- a/pkg/sql/colexec/ordered_synchronizer_tmpl.go +++ b/pkg/sql/colexec/ordered_synchronizer_tmpl.go @@ -141,10 +141,10 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { } heap.Init(o) } - o.output.ResetInternalBatch() + o.resetOutput() outputIdx := 0 o.allocator.PerformOperation(o.output.ColVecs(), func() { - for outputIdx < coldata.BatchSize() { + for outputIdx < o.output.Capacity() { if o.Len() == 0 { // All inputs exhausted. break @@ -203,29 +203,40 @@ func (o *OrderedSynchronizer) Next(ctx context.Context) coldata.Batch { return o.output } +func (o *OrderedSynchronizer) resetOutput() { + var reallocated bool + o.output, reallocated = o.allocator.ResetMaybeReallocate(o.typs, o.output, 1 /* minCapacity */) + if reallocated { + // {{range .}} + // {{range .WidthOverloads}} + o.out_TYPECols = o.out_TYPECols[:0] + // {{end}} + // {{end}} + for i, outVec := range o.output.ColVecs() { + o.outNulls[i] = outVec.Nulls() + switch typeconv.TypeFamilyToCanonicalTypeFamily(o.typs[i].Family()) { + // {{range .}} + case _CANONICAL_TYPE_FAMILY: + switch o.typs[i].Width() { + // {{range .WidthOverloads}} + case _TYPE_WIDTH: + o.outColsMap[i] = len(o.out_TYPECols) + o.out_TYPECols = append(o.out_TYPECols, outVec._TYPE()) + // {{end}} + } + // {{end}} + default: + colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) + } + } + } +} + // Init is part of the Operator interface. func (o *OrderedSynchronizer) Init() { o.inputIndices = make([]int, len(o.inputs)) - o.output = o.allocator.NewMemBatchWithMaxCapacity(o.typs) o.outNulls = make([]*coldata.Nulls, len(o.typs)) o.outColsMap = make([]int, len(o.typs)) - for i, outVec := range o.output.ColVecs() { - o.outNulls[i] = outVec.Nulls() - switch typeconv.TypeFamilyToCanonicalTypeFamily(o.typs[i].Family()) { - // {{range .}} - case _CANONICAL_TYPE_FAMILY: - switch o.typs[i].Width() { - // {{range .WidthOverloads}} - case _TYPE_WIDTH: - o.outColsMap[i] = len(o.out_TYPECols) - o.out_TYPECols = append(o.out_TYPECols, outVec._TYPE()) - // {{end}} - } - // {{end}} - default: - colexecerror.InternalError(fmt.Sprintf("unhandled type %s", o.typs[i])) - } - } for i := range o.inputs { o.inputs[i].Op.Init() } diff --git a/pkg/sql/colexec/relative_rank.eg.go b/pkg/sql/colexec/relative_rank.eg.go index 777debcce98b..cefbfa9e3138 100644 --- a/pkg/sql/colexec/relative_rank.eg.go +++ b/pkg/sql/colexec/relative_rank.eg.go @@ -181,7 +181,7 @@ func (r *percentRankNoPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) - r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithFixedCapacity(append(r.inputTypes, types.Float), coldata.BatchSize()) // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this // rankIncrement to 1 will update r.rank to 1 on the very first tuple (as @@ -381,7 +381,7 @@ func (r *percentRankWithPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) - r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithFixedCapacity(append(r.inputTypes, types.Float), coldata.BatchSize()) // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this // rankIncrement to 1 will update r.rank to 1 on the very first tuple (as @@ -432,7 +432,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -494,7 +494,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -524,7 +524,7 @@ func (r *percentRankWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -690,7 +690,7 @@ func (r *cumeDistNoPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) - r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithFixedCapacity(append(r.inputTypes, types.Float), coldata.BatchSize()) } func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { @@ -736,7 +736,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers @@ -800,7 +800,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -830,7 +830,7 @@ func (r *cumeDistNoPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -993,7 +993,7 @@ func (r *cumeDistWithPartitionOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) - r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithFixedCapacity(append(r.inputTypes, types.Float), coldata.BatchSize()) } func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { @@ -1039,7 +1039,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -1057,7 +1057,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers @@ -1119,7 +1119,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -1149,7 +1149,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -1187,7 +1187,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -1217,7 +1217,7 @@ func (r *cumeDistWithPartitionOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { diff --git a/pkg/sql/colexec/relative_rank_tmpl.go b/pkg/sql/colexec/relative_rank_tmpl.go index dd26601612f3..d477ad1ae09b 100644 --- a/pkg/sql/colexec/relative_rank_tmpl.go +++ b/pkg/sql/colexec/relative_rank_tmpl.go @@ -144,7 +144,7 @@ func _COMPUTE_PARTITIONS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPartitionsSizesCol = r.partitionsState.runningSizes.ColVec(0).Int64() } if r.numTuplesInPartition > 0 { @@ -182,7 +182,7 @@ func _COMPUTE_PEER_GROUPS_SIZES() { // */}} // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) runningPeerGroupsSizesCol = r.peerGroupsState.runningSizes.ColVec(0).Int64() } if r.numPeers > 0 { @@ -300,7 +300,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Init() { int64(float64(r.memoryLimit)*(1.0-usedMemoryLimitFraction)), r.diskQueueCfg, r.fdSemaphore, r.diskAcc, ) - r.output = r.allocator.NewMemBatchWithMaxCapacity(append(r.inputTypes, types.Float)) + r.output = r.allocator.NewMemBatchWithFixedCapacity(append(r.inputTypes, types.Float), coldata.BatchSize()) // {{if .IsPercentRank}} // All rank functions start counting from 1. Before we assign the rank to a // tuple in the batch, we first increment r.rank, so setting this @@ -358,7 +358,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.partitionsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.partitionsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPartitionsSizesCol := r.partitionsState.runningSizes.ColVec(0).Int64() runningPartitionsSizesCol[r.partitionsState.idx] = r.numTuplesInPartition @@ -378,7 +378,7 @@ func (r *_RELATIVE_RANK_STRINGOp) Next(ctx context.Context) coldata.Batch { // TODO(yuzefovich): do not instantiate a new batch here once // spillingQueues actually copy the batches when those are kept // in-memory. - r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithMaxCapacity([]*types.T{types.Int}) + r.peerGroupsState.runningSizes = r.allocator.NewMemBatchWithFixedCapacity([]*types.T{types.Int}, coldata.BatchSize()) } runningPeerGroupsSizesCol := r.peerGroupsState.runningSizes.ColVec(0).Int64() runningPeerGroupsSizesCol[r.peerGroupsState.idx] = r.numPeers diff --git a/pkg/sql/colexec/routers.go b/pkg/sql/colexec/routers.go index 561e18c307b9..83b6b8ba63cb 100644 --- a/pkg/sql/colexec/routers.go +++ b/pkg/sql/colexec/routers.go @@ -412,9 +412,9 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele for toAppend := len(selection); toAppend > 0; { if o.mu.pendingBatch == nil { - o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithMaxCapacity(o.types) + o.mu.pendingBatch = o.mu.unlimitedAllocator.NewMemBatchWithFixedCapacity(o.types, coldata.BatchSize()) } - available := coldata.BatchSize() - o.mu.pendingBatch.Length() + available := o.mu.pendingBatch.Capacity() - o.mu.pendingBatch.Length() numAppended := toAppend if toAppend > available { numAppended = available @@ -435,7 +435,7 @@ func (o *routerOutputOp) addBatch(ctx context.Context, batch coldata.Batch, sele }) newLength := o.mu.pendingBatch.Length() + numAppended o.mu.pendingBatch.SetLength(newLength) - if o.testingKnobs.alwaysFlush || newLength >= coldata.BatchSize() { + if o.testingKnobs.alwaysFlush || newLength >= o.mu.pendingBatch.Capacity() { // The capacity in o.mu.pendingBatch has been filled. err := o.mu.data.enqueue(ctx, o.mu.pendingBatch) if err == nil && o.testingKnobs.addBatchTestInducedErrorCb != nil { diff --git a/pkg/sql/colexec/sort.go b/pkg/sql/colexec/sort.go index 0040946bd143..e8184a36de7e 100644 --- a/pkg/sql/colexec/sort.go +++ b/pkg/sql/colexec/sort.go @@ -249,52 +249,59 @@ const ( // sortEmitting is the third state of the operator, indicating that each call // to Next will return another batch of the sorted data. sortEmitting + // sortDone is the final state of the operator, where it always returns a + // zero batch. + sortDone ) func (p *sortOp) Next(ctx context.Context) coldata.Batch { - switch p.state { - case sortSpooling: - p.input.spool(ctx) - p.state = sortSorting - fallthrough - case sortSorting: - p.sort(ctx) - p.state = sortEmitting - fallthrough - case sortEmitting: - newEmitted := p.emitted + coldata.BatchSize() - if newEmitted > p.input.getNumTuples() { - newEmitted = p.input.getNumTuples() - } - if newEmitted == p.emitted { - return coldata.ZeroBatch - } - - p.resetOutput() - for j := 0; j < len(p.inputTypes); j++ { - // At this point, we have already fully sorted the input. It is ok to do - // this Copy outside of the allocator - the work has been done, but - // theoretically it is possible to hit the limit here (mainly with - // variable-sized types like Bytes). Nonetheless, for performance reasons - // it would be sad to fallback to disk at this point. - p.output.ColVec(j).Copy( - coldata.CopySliceArgs{ - SliceArgs: coldata.SliceArgs{ - Sel: p.order, - Src: p.input.getValues(j), - SrcStartIdx: p.emitted, - SrcEndIdx: newEmitted, + for { + switch p.state { + case sortSpooling: + p.input.spool(ctx) + p.state = sortSorting + case sortSorting: + p.sort(ctx) + p.state = sortEmitting + case sortEmitting: + toEmit := p.input.getNumTuples() - p.emitted + if toEmit == 0 { + p.state = sortDone + continue + } + if toEmit > coldata.BatchSize() { + toEmit = coldata.BatchSize() + } + p.output, _ = p.allocator.ResetMaybeReallocate(p.inputTypes, p.output, toEmit) + newEmitted := p.emitted + toEmit + for j := 0; j < len(p.inputTypes); j++ { + // At this point, we have already fully sorted the input. It is ok to do + // this Copy outside of the allocator - the work has been done, but + // theoretically it is possible to hit the limit here (mainly with + // variable-sized types like Bytes). Nonetheless, for performance reasons + // it would be sad to fallback to disk at this point. + p.output.ColVec(j).Copy( + coldata.CopySliceArgs{ + SliceArgs: coldata.SliceArgs{ + Sel: p.order, + Src: p.input.getValues(j), + SrcStartIdx: p.emitted, + SrcEndIdx: newEmitted, + }, }, - }, - ) + ) + } + p.output.SetLength(toEmit) + p.emitted = newEmitted + return p.output + case sortDone: + return coldata.ZeroBatch + default: + colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", p.state)) + // This code is unreachable, but the compiler cannot infer that. + return nil } - p.output.SetLength(newEmitted - p.emitted) - p.emitted = newEmitted - return p.output } - colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", p.state)) - // This code is unreachable, but the compiler cannot infer that. - return nil } // sort sorts the spooled tuples, so it must be called after spool() has been @@ -394,14 +401,6 @@ func (p *sortOp) sort(ctx context.Context) { } } -func (p *sortOp) resetOutput() { - if p.output == nil { - p.output = p.allocator.NewMemBatchWithMaxCapacity(p.inputTypes) - } else { - p.output.ResetInternalBatch() - } -} - func (p *sortOp) reset(ctx context.Context) { if r, ok := p.input.(resetter); ok { r.reset(ctx) diff --git a/pkg/sql/colexec/sorttopk.go b/pkg/sql/colexec/sorttopk.go index 2962dcb84d18..65bd6b13cb1c 100644 --- a/pkg/sql/colexec/sorttopk.go +++ b/pkg/sql/colexec/sorttopk.go @@ -53,12 +53,15 @@ var _ colexecbase.BufferingInMemoryOperator = &topKSorter{} type topKSortState int const ( - // sortSpooling is the initial state of the operator, where it spools its - // input. + // topKSortSpooling is the initial state of the operator, where it spools + // its input. topKSortSpooling topKSortState = iota - // sortEmitting is the second state of the operator, indicating that each call - // to Next will return another batch of the sorted data. + // topKSortEmitting is the second state of the operator, indicating that + // each call to Next will return another batch of the sorted data. topKSortEmitting + // topKSortDone is the final state of the operator, where it always returns + // a zero batch. + topKSortDone ) type topKSorter struct { @@ -107,17 +110,26 @@ func (t *topKSorter) Init() { } func (t *topKSorter) Next(ctx context.Context) coldata.Batch { - switch t.state { - case topKSortSpooling: - t.spool(ctx) - t.state = topKSortEmitting - fallthrough - case topKSortEmitting: - return t.emit() + for { + switch t.state { + case topKSortSpooling: + t.spool(ctx) + t.state = topKSortEmitting + case topKSortEmitting: + output := t.emit() + if output.Length() == 0 { + t.state = topKSortDone + continue + } + return output + case topKSortDone: + return coldata.ZeroBatch + default: + colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", t.state)) + // This code is unreachable, but the compiler cannot infer that. + return nil + } } - colexecerror.InternalError(fmt.Sprintf("invalid sort state %v", t.state)) - // This code is unreachable, but the compiler cannot infer that. - return nil } // spool reads in the entire input, always storing the top K rows it has seen so @@ -195,16 +207,7 @@ func (t *topKSorter) spool(ctx context.Context) { } } -func (t *topKSorter) resetOutput() { - if t.output == nil { - t.output = t.allocator.NewMemBatchWithMaxCapacity(t.inputTypes) - } else { - t.output.ResetInternalBatch() - } -} - func (t *topKSorter) emit() coldata.Batch { - t.resetOutput() toEmit := t.topK.Length() - t.emitted if toEmit == 0 { // We're done. @@ -213,6 +216,7 @@ func (t *topKSorter) emit() coldata.Batch { if toEmit > coldata.BatchSize() { toEmit = coldata.BatchSize() } + t.output, _ = t.allocator.ResetMaybeReallocate(t.inputTypes, t.output, toEmit) for i := range t.inputTypes { vec := t.output.ColVec(i) // At this point, we have already fully sorted the input. It is ok to do diff --git a/pkg/sql/colexec/spilling_queue.go b/pkg/sql/colexec/spilling_queue.go index a466e2a5f676..bdc03d8334cd 100644 --- a/pkg/sql/colexec/spilling_queue.go +++ b/pkg/sql/colexec/spilling_queue.go @@ -98,7 +98,7 @@ func newSpillingQueue( items: make([]coldata.Batch, itemsLen), diskQueueCfg: cfg, fdSemaphore: fdSemaphore, - dequeueScratch: unlimitedAllocator.NewMemBatchWithMaxCapacity(typs), + dequeueScratch: unlimitedAllocator.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()), diskAcc: diskAcc, } } diff --git a/pkg/sql/colexec/stats_test.go b/pkg/sql/colexec/stats_test.go index 73698de88db3..7e128f419f9e 100644 --- a/pkg/sql/colexec/stats_test.go +++ b/pkg/sql/colexec/stats_test.go @@ -123,28 +123,26 @@ func TestVectorizedStatsCollector(t *testing.T) { []*VectorizedStatsCollector{leftInput, rightInput}, ) - // The inputs are identical, so the merge joiner should output nBatches - // batches with each having coldata.BatchSize() tuples. + // The inputs are identical, so the merge joiner should output + // nBatches x coldata.BatchSize() tuples. mjStatsCollector.Init() - batchCount := 0 + batchCount, tupleCount := 0, 0 for { b := mjStatsCollector.Next(context.Background()) if b.Length() == 0 { break } - require.Equal(t, coldata.BatchSize(), b.Length()) batchCount++ + tupleCount += b.Length() } mjStatsCollector.finalizeStats() - require.Equal(t, nBatches, batchCount) - require.Equal(t, nBatches, int(mjStatsCollector.NumBatches)) require.Equal(t, nBatches*coldata.BatchSize(), int(mjStatsCollector.NumTuples)) // Two inputs are advancing the time source for a total of 2 * nBatches // advances, but these do not count towards merge joiner execution time. // Merge joiner advances the time on its every non-empty batch totaling - // nBatches advances that should be accounted for in stats. - require.Equal(t, time.Duration(nBatches), mjStatsCollector.Time) + // batchCount advances that should be accounted for in stats. + require.Equal(t, time.Duration(batchCount), mjStatsCollector.Time) } } diff --git a/pkg/sql/colexec/unordered_distinct.go b/pkg/sql/colexec/unordered_distinct.go index d7059c8ce1ea..c91087e7b911 100644 --- a/pkg/sql/colexec/unordered_distinct.go +++ b/pkg/sql/colexec/unordered_distinct.go @@ -44,7 +44,7 @@ func NewUnorderedDistinct( OneInputNode: NewOneInputNode(input), allocator: allocator, ht: ht, - output: allocator.NewMemBatchWithMaxCapacity(typs), + typs: typs, } } @@ -60,6 +60,7 @@ type unorderedDistinct struct { allocator *colmem.Allocator ht *hashTable + typs []*types.T buildFinished bool distinctCount int @@ -75,7 +76,6 @@ func (op *unorderedDistinct) Init() { } func (op *unorderedDistinct) Next(ctx context.Context) coldata.Batch { - op.output.ResetInternalBatch() // First, build the hash table and populate the selection vector that // includes only distinct tuples. if !op.buildFinished { @@ -86,11 +86,15 @@ func (op *unorderedDistinct) Next(ctx context.Context) coldata.Batch { // tuples, as a result, we will be simply returning all buffered tuples. op.distinctCount = op.ht.vals.Length() } + if op.outputBatchStart == op.distinctCount { + return coldata.ZeroBatch + } + op.output, _ = op.allocator.ResetMaybeReallocate(op.typs, op.output, op.distinctCount-op.outputBatchStart) - // Create and return the next batch of input to a maximum size of - // coldata.BatchSize(). + // Create and return the next batch of input to a maximum size equal to the + // capacity of the output batch. nSelected := 0 - batchEnd := op.outputBatchStart + coldata.BatchSize() + batchEnd := op.outputBatchStart + op.output.Capacity() if batchEnd > op.distinctCount { batchEnd = op.distinctCount } diff --git a/pkg/sql/colexec/utils.go b/pkg/sql/colexec/utils.go index 6cdd605bd904..d8e233c66b5a 100644 --- a/pkg/sql/colexec/utils.go +++ b/pkg/sql/colexec/utils.go @@ -91,7 +91,7 @@ func newPartitionerToOperator( return &partitionerToOperator{ partitioner: partitioner, partitionIdx: partitionIdx, - batch: allocator.NewMemBatchWithMaxCapacity(types), + batch: allocator.NewMemBatchWithFixedCapacity(types, coldata.BatchSize()), } } @@ -118,6 +118,12 @@ func (p *partitionerToOperator) Next(ctx context.Context) coldata.Batch { return p.batch } +// newAppendOnlyBufferedBatch returns a new appendOnlyBufferedBatch that has +// initial zero capacity and could grow arbitrarily large with append() method. +// It is intended to be used by the operators that need to buffer unknown +// number of tuples. +// TODO(yuzefovich): consider whether it is beneficial to start out with +// non-zero capacity. func newAppendOnlyBufferedBatch( allocator *colmem.Allocator, typs []*types.T, ) *appendOnlyBufferedBatch { diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 93688bff8d34..fa0179624eef 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -249,12 +249,31 @@ type cFetcher struct { timestampCol []apd.Decimal } + typs []*types.T + allocator *colmem.Allocator + // adapter is a utility struct that helps with memory accounting. adapter struct { - ctx context.Context - allocator *colmem.Allocator - batch coldata.Batch - err error + ctx context.Context + batch coldata.Batch + err error + } +} + +const cFetcherBatchMinCapacity = 1 + +func (rf *cFetcher) resetBatch(timestampOutputIdx int) { + var reallocated bool + rf.machine.batch, reallocated = rf.allocator.ResetMaybeReallocate( + rf.typs, rf.machine.batch, cFetcherBatchMinCapacity, + ) + if reallocated { + rf.machine.colvecs = rf.machine.batch.ColVecs() + // If the fetcher is requested to produce a timestamp column, pull out the + // column as a decimal and save it. + if timestampOutputIdx != noTimestampColumn { + rf.machine.timestampCol = rf.machine.colvecs[timestampOutputIdx].Decimal() + } } } @@ -268,7 +287,7 @@ func (rf *cFetcher) Init( lockStr descpb.ScanLockingStrength, tables ...row.FetcherTableArgs, ) error { - rf.adapter.allocator = allocator + rf.allocator = allocator if len(tables) == 0 { return errors.AssertionFailedf("no tables to fetch from") } @@ -302,9 +321,9 @@ func (rf *cFetcher) Init( timestampOutputIdx: noTimestampColumn, } - typs := make([]*types.T, len(colDescriptors)) - for i := range typs { - typs[i] = colDescriptors[i].Type + rf.typs = make([]*types.T, len(colDescriptors)) + for i := range rf.typs { + rf.typs[i] = colDescriptors[i].Type } var err error @@ -328,14 +347,7 @@ func (rf *cFetcher) Init( } sort.Ints(table.neededColsList) - rf.machine.batch = allocator.NewMemBatchWithMaxCapacity(typs) - rf.machine.colvecs = rf.machine.batch.ColVecs() - // If the fetcher is requested to produce a timestamp column, pull out the - // column as a decimal and save it. - if table.timestampOutputIdx != noTimestampColumn { - rf.machine.timestampCol = rf.machine.colvecs[table.timestampOutputIdx].Decimal() - } - + rf.resetBatch(table.timestampOutputIdx) table.knownPrefixLength = len(sqlbase.MakeIndexKeyPrefix(codec, table.desc, table.index.ID)) var indexColumnIDs []descpb.ColumnID @@ -611,7 +623,7 @@ const debugState = false // NextBatch is nextBatch with the addition of memory accounting. func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { rf.adapter.ctx = ctx - rf.adapter.allocator.PerformOperation( + rf.allocator.PerformOperation( rf.machine.colvecs, rf.nextAdapter, ) @@ -672,7 +684,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.machine.state[0] = stateDecodeFirstKVOfRow case stateResetBatch: - rf.machine.batch.ResetInternalBatch() + rf.resetBatch(rf.table.timestampOutputIdx) rf.shiftState() case stateDecodeFirstKVOfRow: // Reset MVCC metadata for the table, since this is the first KV of a row. @@ -904,7 +916,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { } rf.machine.rowIdx++ rf.shiftState() - if rf.machine.rowIdx >= coldata.BatchSize() { + if rf.machine.rowIdx >= rf.machine.batch.Capacity() { rf.pushState(stateResetBatch) rf.machine.batch.SetLength(rf.machine.rowIdx) rf.machine.rowIdx = 0 diff --git a/pkg/sql/colflow/colrpc/inbox.go b/pkg/sql/colflow/colrpc/inbox.go index 7e8686644eb2..f8161502c71e 100644 --- a/pkg/sql/colflow/colrpc/inbox.go +++ b/pkg/sql/colflow/colrpc/inbox.go @@ -47,6 +47,7 @@ type Inbox struct { colexecbase.ZeroInputNode typs []*types.T + allocator *colmem.Allocator converter *colserde.ArrowBatchConverter serializer *colserde.RecordBatchSerializer @@ -120,6 +121,7 @@ func NewInbox( } i := &Inbox{ typs: typs, + allocator: allocator, converter: c, serializer: s, streamID: streamID, @@ -130,7 +132,6 @@ func NewInbox( flowCtx: ctx, } i.scratch.data = make([]*array.Data, len(typs)) - i.scratch.b = allocator.NewMemBatchWithMaxCapacity(typs) return i, nil } @@ -300,6 +301,11 @@ func (i *Inbox) Next(ctx context.Context) coldata.Batch { if err := i.serializer.Deserialize(&i.scratch.data, m.Data.RawBytes); err != nil { colexecerror.InternalError(err) } + i.scratch.b, _ = i.allocator.ResetMaybeReallocate( + // We don't support type-less schema, so len(i.scratch.data) is + // always at least 1. + i.typs, i.scratch.b, i.scratch.data[0].Len(), + ) if err := i.converter.ArrowToBatch(i.scratch.data, i.scratch.b); err != nil { colexecerror.InternalError(err) } diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index 93c813e1ecdc..96dca8829ffe 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -89,14 +89,10 @@ func NewAllocator( } } -// NewMemBatchWithMaxCapacity allocates a new in-memory coldata.Batch of -// coldata.BatchSize() capacity. -func (a *Allocator) NewMemBatchWithMaxCapacity(typs []*types.T) coldata.Batch { - return a.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()) -} - -// NewMemBatchWithFixedCapacity allocates a new in-memory coldata.Batch with -// the given capacity. +// NewMemBatchWithFixedCapacity allocates a new in-memory coldata.Batch with the +// given vector capacity. +// Note: consider whether you want the dynamic batch size behavior (in which +// case you should be using ResetMaybeReallocate). func (a *Allocator) NewMemBatchWithFixedCapacity(typs []*types.T, capacity int) coldata.Batch { estimatedMemoryUsage := selVectorSize(capacity) + int64(EstimateBatchSizeBytes(typs, capacity)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { @@ -105,15 +101,58 @@ func (a *Allocator) NewMemBatchWithFixedCapacity(typs []*types.T, capacity int) return coldata.NewMemBatchWithCapacity(typs, capacity, a.factory) } +// NewMemBatchWithMaxCapacity is a convenience shortcut of +// NewMemBatchWithFixedCapacity with capacity=coldata.BatchSize() and should +// only be used in tests (this is enforced by a linter). +func (a *Allocator) NewMemBatchWithMaxCapacity(typs []*types.T) coldata.Batch { + return a.NewMemBatchWithFixedCapacity(typs, coldata.BatchSize()) +} + // NewMemBatchNoCols creates a "skeleton" of new in-memory coldata.Batch. It // allocates memory for the selection vector but does *not* allocate any memory // for the column vectors - those will have to be added separately. -func (a *Allocator) NewMemBatchNoCols(types []*types.T, capacity int) coldata.Batch { +func (a *Allocator) NewMemBatchNoCols(typs []*types.T, capacity int) coldata.Batch { estimatedMemoryUsage := selVectorSize(capacity) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { colexecerror.InternalError(err) } - return coldata.NewMemBatchNoCols(types, capacity) + return coldata.NewMemBatchNoCols(typs, capacity) +} + +// ResetMaybeReallocate returns a batch that is guaranteed to be in a "reset" +// state (meaning it is ready to be used) and to have the capacity of at least +// minCapacity. The method will grow the allocated capacity of the batch +// exponentially (possibly incurring a reallocation), until the batch reaches +// coldata.BatchSize(). +// Note: the method assumes that minCapacity is at least 1 and will "truncate" +// minCapacity if it is larger than coldata.BatchSize(). +func (a *Allocator) ResetMaybeReallocate( + typs []*types.T, oldBatch coldata.Batch, minCapacity int, +) (newBatch coldata.Batch, reallocated bool) { + if minCapacity < 1 { + colexecerror.InternalError(errors.AssertionFailedf("invalid minCapacity %d", minCapacity)) + } + if minCapacity > coldata.BatchSize() { + minCapacity = coldata.BatchSize() + } + reallocated = true + if oldBatch == nil { + newBatch = a.NewMemBatchWithFixedCapacity(typs, minCapacity) + } else if oldBatch.Capacity() < coldata.BatchSize() { + newCapacity := oldBatch.Capacity() * 2 + if newCapacity < minCapacity { + newCapacity = minCapacity + } + if newCapacity > coldata.BatchSize() { + newCapacity = coldata.BatchSize() + } + newBatch = a.NewMemBatchWithFixedCapacity(typs, newCapacity) + } else { + reallocated = false + oldBatch.ResetInternalBatch() + newBatch = oldBatch + } + return newBatch, reallocated } // RetainBatch adds the size of the batch to the memory account. This shouldn't @@ -162,8 +201,8 @@ func (a *Allocator) ReleaseBatch(b coldata.Batch) { } // NewMemColumn returns a new coldata.Vec of the desired capacity. -// NOTE: consider whether you should be using MaybeAppendColumn or -// NewMemBatchWith* methods or colexec.DynamicBatchSizeHelper struct. +// NOTE: consider whether you should be using MaybeAppendColumn, +// NewMemBatchWith*, or ResetMaybeReallocate methods. func (a *Allocator) NewMemColumn(t *types.T, capacity int) coldata.Vec { estimatedMemoryUsage := int64(EstimateBatchSizeBytes([]*types.T{t}, capacity)) if err := a.acc.Grow(a.ctx, estimatedMemoryUsage); err != nil { diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 981a03013ad5..c4fab50f5022 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1594,6 +1594,42 @@ func TestLint(t *testing.T) { } }) + t.Run("TestVectorizedDynamicBatches", func(t *testing.T) { + // t.Parallel() // Disabled due to CI not parsing failure from parallel tests correctly. Can be re-enabled on Go 1.15 (see: https://github.com/golang/go/issues/38458). + cmd, stderr, filter, err := dirCmd( + pkgDir, + "git", + "grep", + "-nE", + // We prohibit usage of colmem.Allocator.NewMemBatchWithMaxCapacity + // in order to remind us to think whether we want the dynamic batch + // size behavior or not. + fmt.Sprintf(`\.NewMemBatchWithMaxCapacity\(`), + "--", + "sql/col*", + ":!sql/col*_test.go", + ) + if err != nil { + t.Fatal(err) + } + + if err := cmd.Start(); err != nil { + t.Fatal(err) + } + + if err := stream.ForEach(filter, func(s string) { + t.Errorf("\n%s <- forbidden; either use ResetMaybeReallocate or NewMemBatchWithFixedCapacity", s) + }); err != nil { + t.Error(err) + } + + if err := cmd.Wait(); err != nil { + if out := stderr.String(); len(out) > 0 { + t.Fatalf("err=%s, stderr=%s", err, out) + } + } + }) + t.Run("TestVectorizedAppendColumn", func(t *testing.T) { // t.Parallel() // Disabled due to CI not parsing failure from parallel tests correctly. Can be re-enabled on Go 1.15 (see: https://github.com/golang/go/issues/38458). cmd, stderr, filter, err := dirCmd(