diff --git a/pkg/sql/colmem/BUILD.bazel b/pkg/sql/colmem/BUILD.bazel index 1f38a68b997d..3f7b746cd173 100644 --- a/pkg/sql/colmem/BUILD.bazel +++ b/pkg/sql/colmem/BUILD.bazel @@ -22,9 +22,12 @@ go_library( go_test( name = "colmem_test", size = "small", - srcs = ["allocator_test.go"], + srcs = [ + "allocator_test.go", + "reset_maybe_reallocate_test.go", + ], + embed = [":colmem"], deps = [ - ":colmem", "//pkg/col/coldata", "//pkg/col/coldataext", "//pkg/settings/cluster", diff --git a/pkg/sql/colmem/allocator.go b/pkg/sql/colmem/allocator.go index 64d492a3b92e..b9fdf1cbd1ac 100644 --- a/pkg/sql/colmem/allocator.go +++ b/pkg/sql/colmem/allocator.go @@ -157,7 +157,7 @@ func (a *Allocator) ResetBatch(batch coldata.Batch) { a.ReleaseMemory(batch.ResetInternalBatch()) } -// ResetMaybeReallocate returns a batch that is guaranteed to be in a "reset" +// 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 // 1. minDesiredCapacity is a hint about the capacity of the returned batch // (subject to the memory limit). @@ -180,8 +180,7 @@ func (a *Allocator) ResetBatch(batch coldata.Batch) { // old batch. // Note: the method assumes that minDesiredCapacity is at least 0 and will clamp // minDesiredCapacity to be between 1 and coldata.BatchSize() inclusive. -// TODO(yuzefovich): unexport this method. -func (a *Allocator) ResetMaybeReallocate( +func (a *Allocator) resetMaybeReallocate( typs []*types.T, oldBatch coldata.Batch, minDesiredCapacity int, @@ -234,7 +233,7 @@ func (a *Allocator) ResetMaybeReallocate( return newBatch, reallocated, oldBatchReachedMemSize } -// ResetMaybeReallocateNoMemLimit is the same as ResetMaybeReallocate when +// ResetMaybeReallocateNoMemLimit is the same as resetMaybeReallocate when // MaxInt64 is used as the maxBatchMemSize argument and the desired capacity is // sufficient. This should be used by the callers that know exactly the capacity // they need and have no control over that number. It is guaranteed that the @@ -243,7 +242,7 @@ func (a *Allocator) ResetMaybeReallocate( func (a *Allocator) ResetMaybeReallocateNoMemLimit( typs []*types.T, oldBatch coldata.Batch, requiredCapacity int, ) (newBatch coldata.Batch, reallocated bool) { - newBatch, reallocated, _ = a.ResetMaybeReallocate(typs, oldBatch, requiredCapacity, math.MaxInt64, true /* desiredCapacitySufficient */) + newBatch, reallocated, _ = a.resetMaybeReallocate(typs, oldBatch, requiredCapacity, math.MaxInt64, true /* desiredCapacitySufficient */) return newBatch, reallocated } @@ -544,7 +543,7 @@ func GetFixedSizeTypeSize(t *types.T) (size int64) { // factor of two, then that batch is discarded, and the capacity will never // exceed half of the capacity of the discarded batch; // - if the memory limit is not reached, then the behavior of the dynamic growth -// of the capacity provided by Allocator.ResetMaybeReallocate is still +// of the capacity provided by Allocator.resetMaybeReallocate is still // applicable (i.e. the capacities will grow exponentially until // coldata.BatchSize()). // @@ -631,7 +630,7 @@ func (h *AccountingHelper) ResetMaybeReallocate( } // By default, assume that the number of tuples to be set is sufficient and // ask for it. If that number is unknown, we'll rely on the - // Allocator.ResetMaybeReallocate method to provide the dynamically-growing + // Allocator.resetMaybeReallocate method to provide the dynamically-growing // batches. minDesiredCapacity := tuplesToBeSet desiredCapacitySufficient := tuplesToBeSet > 0 @@ -645,13 +644,13 @@ func (h *AccountingHelper) ResetMaybeReallocate( desiredCapacitySufficient = true } var oldBatchReachedMemSize bool - newBatch, reallocated, oldBatchReachedMemSize = h.allocator.ResetMaybeReallocate( + newBatch, reallocated, oldBatchReachedMemSize = h.allocator.resetMaybeReallocate( typs, oldBatch, minDesiredCapacity, h.memoryLimit, desiredCapacitySufficient, ) if oldBatchReachedMemSize && h.maxCapacity == 0 { // The old batch has just reached the memory size for the first time, so // we memorize the maximum capacity. Note that this is not strictly - // necessary to do (since Allocator.ResetMaybeReallocate would never + // necessary to do (since Allocator.resetMaybeReallocate would never // allocate a new batch from now on), but it makes things more clear and // allows us to avoid computing the memory size of the batch on each // call. @@ -753,7 +752,7 @@ func (h *SetAccountingHelper) ResetMaybeReallocate( newBatch, reallocated = h.helper.ResetMaybeReallocate(typs, oldBatch, tuplesToBeSet) h.curCapacity = newBatch.Capacity() if reallocated && !h.allFixedLength { - // Allocator.ResetMaybeReallocate has released the precise memory + // Allocator.resetMaybeReallocate has released the precise memory // footprint of the old batch and has accounted for the estimated // footprint of the new batch. This means that we need to update our // internal memory tracking state to those estimates. diff --git a/pkg/sql/colmem/allocator_test.go b/pkg/sql/colmem/allocator_test.go index bb10514d2b05..1c1389c72c3b 100644 --- a/pkg/sql/colmem/allocator_test.go +++ b/pkg/sql/colmem/allocator_test.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -97,94 +96,6 @@ func TestMaybeAppendColumn(t *testing.T) { }) } -func TestResetMaybeReallocate(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - st := cluster.MakeTestingClusterSettings() - testMemMonitor := execinfra.NewTestMemMonitor(ctx, st) - defer testMemMonitor.Stop(ctx) - memAcc := testMemMonitor.MakeBoundAccount() - defer memAcc.Close(ctx) - evalCtx := eval.MakeTestingEvalContext(st) - testColumnFactory := coldataext.NewExtendedColumnFactory(&evalCtx) - testAllocator := colmem.NewAllocator(ctx, &memAcc, testColumnFactory) - - t.Run("ResettingBehavior", func(t *testing.T) { - if coldata.BatchSize() == 1 { - skip.IgnoreLint(t, "the test assumes coldata.BatchSize() is at least 2") - } - - var b coldata.Batch - typs := []*types.T{types.Bytes} - - // Allocate a new batch and modify it. - b, _, _ = testAllocator.ResetMaybeReallocate(typs, b, coldata.BatchSize(), math.MaxInt64, false /* desiredCapacitySufficient */) - b.SetSelection(true) - b.Selection()[0] = 1 - b.ColVec(0).Bytes().Set(1, []byte("foo")) - - oldBatch := b - b, _, _ = testAllocator.ResetMaybeReallocate(typs, b, coldata.BatchSize(), math.MaxInt64, false /* desiredCapacitySufficient */) - // We should have used the same batch, and now it should be in a "reset" - // state. - require.Equal(t, oldBatch, b) - require.Nil(t, b.Selection()) - // We should be able to set in the Bytes vector using an arbitrary - // position since the vector should have been reset. - require.NotPanics(t, func() { b.ColVec(0).Bytes().Set(0, []byte("bar")) }) - }) - - t.Run("LimitingByMemSize", func(t *testing.T) { - if coldata.BatchSize() == 1 { - skip.IgnoreLint(t, "the test assumes coldata.BatchSize() is at least 2") - } - - var b coldata.Batch - typs := []*types.T{types.Int} - const minDesiredCapacity = 2 - const smallMemSize = 0 - const largeMemSize = math.MaxInt64 - - // Allocate a batch with smaller capacity. - smallBatch := testAllocator.NewMemBatchWithFixedCapacity(typs, minDesiredCapacity/2) - - // Allocate a new batch attempting to use the batch with too small of a - // capacity - new batch should **not** be allocated because the memory - // limit is already exceeded. - b, _, _ = testAllocator.ResetMaybeReallocate(typs, smallBatch, minDesiredCapacity, smallMemSize, false /* desiredCapacitySufficient */) - require.Equal(t, smallBatch, b) - require.Equal(t, minDesiredCapacity/2, b.Capacity()) - - oldBatch := b - - // Reset the batch asking for the same small desired capacity when it is - // sufficient - the same batch should be returned. - b, _, _ = testAllocator.ResetMaybeReallocate(typs, b, minDesiredCapacity/2, smallMemSize, true /* desiredCapacitySufficient */) - require.Equal(t, smallBatch, b) - require.Equal(t, minDesiredCapacity/2, b.Capacity()) - - // Reset the batch and confirm that a new batch is allocated because we - // have given larger memory limit. - b, _, _ = testAllocator.ResetMaybeReallocate(typs, b, minDesiredCapacity, largeMemSize, false /* desiredCapacitySufficient */) - require.NotEqual(t, oldBatch, b) - require.Equal(t, minDesiredCapacity, b.Capacity()) - - if coldata.BatchSize() >= minDesiredCapacity*2 { - // Now reset the batch with large memory limit - we should get a new - // batch with the double capacity. - // - // ResetMaybeReallocate truncates the capacity at - // coldata.BatchSize(), so we run this part of the test only when - // doubled capacity will not be truncated. - b, _, _ = testAllocator.ResetMaybeReallocate(typs, b, minDesiredCapacity, largeMemSize, false /* desiredCapacitySufficient */) - require.NotEqual(t, oldBatch, b) - require.Equal(t, 2*minDesiredCapacity, b.Capacity()) - } - }) -} - func TestPerformAppend(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/colmem/reset_maybe_reallocate_test.go b/pkg/sql/colmem/reset_maybe_reallocate_test.go new file mode 100644 index 000000000000..ba37f56dabea --- /dev/null +++ b/pkg/sql/colmem/reset_maybe_reallocate_test.go @@ -0,0 +1,116 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colmem + +import ( + "context" + "math" + "testing" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestResetMaybeReallocate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + st := cluster.MakeTestingClusterSettings() + testMemMonitor := execinfra.NewTestMemMonitor(ctx, st) + defer testMemMonitor.Stop(ctx) + memAcc := testMemMonitor.MakeBoundAccount() + defer memAcc.Close(ctx) + evalCtx := eval.MakeTestingEvalContext(st) + testColumnFactory := coldataext.NewExtendedColumnFactory(&evalCtx) + testAllocator := NewAllocator(ctx, &memAcc, testColumnFactory) + + t.Run("ResettingBehavior", func(t *testing.T) { + if coldata.BatchSize() == 1 { + skip.IgnoreLint(t, "the test assumes coldata.BatchSize() is at least 2") + } + + var b coldata.Batch + typs := []*types.T{types.Bytes} + + // Allocate a new batch and modify it. + b, _, _ = testAllocator.resetMaybeReallocate(typs, b, coldata.BatchSize(), math.MaxInt64, false /* desiredCapacitySufficient */) + b.SetSelection(true) + b.Selection()[0] = 1 + b.ColVec(0).Bytes().Set(1, []byte("foo")) + + oldBatch := b + b, _, _ = testAllocator.resetMaybeReallocate(typs, b, coldata.BatchSize(), math.MaxInt64, false /* desiredCapacitySufficient */) + // We should have used the same batch, and now it should be in a "reset" + // state. + require.Equal(t, oldBatch, b) + require.Nil(t, b.Selection()) + // We should be able to set in the Bytes vector using an arbitrary + // position since the vector should have been reset. + require.NotPanics(t, func() { b.ColVec(0).Bytes().Set(0, []byte("bar")) }) + }) + + t.Run("LimitingByMemSize", func(t *testing.T) { + if coldata.BatchSize() == 1 { + skip.IgnoreLint(t, "the test assumes coldata.BatchSize() is at least 2") + } + + var b coldata.Batch + typs := []*types.T{types.Int} + const minDesiredCapacity = 2 + const smallMemSize = 0 + const largeMemSize = math.MaxInt64 + + // Allocate a batch with smaller capacity. + smallBatch := testAllocator.NewMemBatchWithFixedCapacity(typs, minDesiredCapacity/2) + + // Allocate a new batch attempting to use the batch with too small of a + // capacity - new batch should **not** be allocated because the memory + // limit is already exceeded. + b, _, _ = testAllocator.resetMaybeReallocate(typs, smallBatch, minDesiredCapacity, smallMemSize, false /* desiredCapacitySufficient */) + require.Equal(t, smallBatch, b) + require.Equal(t, minDesiredCapacity/2, b.Capacity()) + + oldBatch := b + + // Reset the batch asking for the same small desired capacity when it is + // sufficient - the same batch should be returned. + b, _, _ = testAllocator.resetMaybeReallocate(typs, b, minDesiredCapacity/2, smallMemSize, true /* desiredCapacitySufficient */) + require.Equal(t, smallBatch, b) + require.Equal(t, minDesiredCapacity/2, b.Capacity()) + + // Reset the batch and confirm that a new batch is allocated because we + // have given larger memory limit. + b, _, _ = testAllocator.resetMaybeReallocate(typs, b, minDesiredCapacity, largeMemSize, false /* desiredCapacitySufficient */) + require.NotEqual(t, oldBatch, b) + require.Equal(t, minDesiredCapacity, b.Capacity()) + + if coldata.BatchSize() >= minDesiredCapacity*2 { + // Now reset the batch with large memory limit - we should get a new + // batch with the double capacity. + // + // resetMaybeReallocate truncates the capacity at + // coldata.BatchSize(), so we run this part of the test only when + // doubled capacity will not be truncated. + b, _, _ = testAllocator.resetMaybeReallocate(typs, b, minDesiredCapacity, largeMemSize, false /* desiredCapacitySufficient */) + require.NotEqual(t, oldBatch, b) + require.Equal(t, 2*minDesiredCapacity, b.Capacity()) + } + }) +}