Skip to content

Commit

Permalink
Merge pull request #123441 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-24.1-123402

release-24.1: rowcontainer: fix possible nil pointer
  • Loading branch information
yuzefovich committed May 2, 2024
2 parents 9400571 + 8d5d832 commit 0d7d1ec
Show file tree
Hide file tree
Showing 6 changed files with 47 additions and 23 deletions.
14 changes: 14 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/tsvector
Original file line number Diff line number Diff line change
Expand Up @@ -476,3 +476,17 @@ FROM
[1, 2, 3]
[1, 2, 3]
[1, 2, 3]

# Regression test for hitting a nil pointer in the row-by-row engine when
# attempting to close uninitialized disk row container (#123141).
statement ok
SET distsql_workmem = '2B';
SET vectorize = off;

statement error pgcode 0A000 unimplemented: can't order by column type TSQUERY
WITH cte (col) AS (VALUES ('foo':::TSQUERY), ('bar':::TSQUERY))
SELECT count(*) FROM cte AS cte1 JOIN cte AS cte2 ON cte1.col = cte2.col;

statement ok
RESET vectorize;
RESET distsql_workmem;
36 changes: 22 additions & 14 deletions pkg/sql/rowcontainer/disk_row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,22 +97,23 @@ var _ DeDupingRowContainer = &DiskRowContainer{}
// - ordering is the output ordering; the order in which rows should be sorted.
// - e is the underlying store that rows are stored on.
func MakeDiskRowContainer(
ctx context.Context,
diskMonitor *mon.BytesMonitor,
typs []*types.T,
ordering colinfo.ColumnOrdering,
e diskmap.Factory,
) (DiskRowContainer, error) {
diskMap := e.NewSortedDiskMap()
d := DiskRowContainer{
diskMap: diskMap,
diskAcc: diskMonitor.MakeBoundAccount(),
types: typs,
ordering: ordering,
diskMonitor: diskMonitor,
engine: e,
datumAlloc: &tree.DatumAlloc{},
diskMap: diskMap,
diskAcc: diskMonitor.MakeBoundAccount(),
bufferedRows: diskMap.NewBatchWriter(),
types: typs,
ordering: ordering,
diskMonitor: diskMonitor,
engine: e,
datumAlloc: &tree.DatumAlloc{},
}
d.bufferedRows = d.diskMap.NewBatchWriter()

// The ordering is specified for a subset of the columns. These will be
// encoded as a key in the given order according to the given direction so
Expand Down Expand Up @@ -143,6 +144,9 @@ func MakeDiskRowContainer(
d.encodings[i] = rowenc.EncodingDirToDatumEncoding(orderInfo.Direction)
switch t := typs[orderInfo.ColIdx]; t.Family() {
case types.TSQueryFamily, types.TSVectorFamily:
// Ensure to close the container since we're not returning it to the
// caller.
d.Close(ctx)
return DiskRowContainer{}, unimplemented.NewWithIssueDetailf(
92165, "", "can't order by column type %s", t.SQLStringForError(),
)
Expand Down Expand Up @@ -333,7 +337,7 @@ func (d *DiskRowContainer) Sort(context.Context) {}
func (d *DiskRowContainer) Reorder(ctx context.Context, ordering colinfo.ColumnOrdering) error {
// We need to create a new DiskRowContainer since its ordering can only be
// changed at initialization.
newContainer, err := MakeDiskRowContainer(d.diskMonitor, d.types, ordering, d.engine)
newContainer, err := MakeDiskRowContainer(ctx, d.diskMonitor, d.types, ordering, d.engine)
if err != nil {
return err
}
Expand Down Expand Up @@ -395,11 +399,15 @@ func (d *DiskRowContainer) UnsafeReset(ctx context.Context) error {

// Close is part of the SortableRowContainer interface.
func (d *DiskRowContainer) Close(ctx context.Context) {
// We can ignore the error here because the flushed data is immediately cleared
// in the following Close.
_ = d.bufferedRows.Close(ctx)
d.diskMap.Close(ctx)
d.diskAcc.Close(ctx)
if d.diskMap != nil {
// diskMap and bufferedRows could be nil in some error paths.

// We can ignore the error here because the flushed data is immediately
// cleared in the following Close.
_ = d.bufferedRows.Close(ctx)
d.diskMap.Close(ctx)
}
d.diskAcc.Close(ctx) // diskAcc is never nil
}

// diskRowIterator iterates over the rows in a DiskRowContainer.
Expand Down
13 changes: 7 additions & 6 deletions pkg/sql/rowcontainer/disk_row_container_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func TestDiskRowContainer(t *testing.T) {
}
row := randgen.RandEncDatumRowOfTypes(rng, typs)
func() {
d, _ := MakeDiskRowContainer(diskMonitor, typs, ordering, tempEngine)
d, _ := MakeDiskRowContainer(ctx, diskMonitor, typs, ordering, tempEngine)
defer d.Close(ctx)
if err := d.AddRow(ctx, row); err != nil {
t.Fatal(err)
Expand Down Expand Up @@ -241,7 +241,7 @@ func TestDiskRowContainer(t *testing.T) {
types := randgen.RandSortingTypes(rng, numCols)
rows := randgen.RandEncDatumRowsOfTypes(rng, numRows, types)
func() {
d, _ := MakeDiskRowContainer(diskMonitor, types, ordering, tempEngine)
d, _ := MakeDiskRowContainer(ctx, diskMonitor, types, ordering, tempEngine)
defer d.Close(ctx)
for i := 0; i < len(rows); i++ {
if err := d.AddRow(ctx, rows[i]); err != nil {
Expand Down Expand Up @@ -323,7 +323,7 @@ func TestDiskRowContainer(t *testing.T) {
// Use random types and random rows.
types := randgen.RandSortingTypes(rng, numCols)
numRows, rows := makeUniqueRows(t, &evalCtx, rng, numRows, types, ordering)
d, _ := MakeDiskRowContainer(diskMonitor, types, ordering, tempEngine)
d, _ := MakeDiskRowContainer(ctx, diskMonitor, types, ordering, tempEngine)
defer d.Close(ctx)
d.DoDeDuplicate()
addRowsRepeatedly := func() {
Expand Down Expand Up @@ -365,7 +365,7 @@ func TestDiskRowContainer(t *testing.T) {
types := randgen.RandSortingTypes(rng, numCols)
rows := randgen.RandEncDatumRowsOfTypes(rng, numRows, types)
// There are no ordering columns when using the numberedRowIterator.
d, _ := MakeDiskRowContainer(diskMonitor, types, nil, tempEngine)
d, _ := MakeDiskRowContainer(ctx, diskMonitor, types, nil, tempEngine)
defer d.Close(ctx)
for i := 0; i < numRows; i++ {
require.NoError(t, d.AddRow(ctx, rows[i]))
Expand Down Expand Up @@ -446,6 +446,7 @@ func TestDiskRowContainerDiskFull(t *testing.T) {
monitor.Start(ctx, nil, mon.NewStandaloneBudget(0 /* capacity */))

d, _ := MakeDiskRowContainer(
ctx,
monitor,
[]*types.T{types.Int},
colinfo.ColumnOrdering{colinfo.ColumnOrderInfo{ColIdx: 0, Direction: encoding.Ascending}},
Expand Down Expand Up @@ -478,7 +479,7 @@ func TestDiskRowContainerFinalIterator(t *testing.T) {
diskMonitor.Start(ctx, nil /* pool */, mon.NewStandaloneBudget(math.MaxInt64))
defer diskMonitor.Stop(ctx)

d, _ := MakeDiskRowContainer(diskMonitor, types.OneIntCol, nil /* ordering */, tempEngine)
d, _ := MakeDiskRowContainer(ctx, diskMonitor, types.OneIntCol, nil /* ordering */, tempEngine)
defer d.Close(ctx)

const numCols = 1
Expand Down Expand Up @@ -598,7 +599,7 @@ func TestDiskRowContainerUnsafeReset(t *testing.T) {
monitor := getDiskMonitor(st)
monitor.Start(ctx, nil, mon.NewStandaloneBudget(math.MaxInt64))

d, _ := MakeDiskRowContainer(monitor, types.OneIntCol, nil /* ordering */, tempEngine)
d, _ := MakeDiskRowContainer(ctx, monitor, types.OneIntCol, nil /* ordering */, tempEngine)
defer d.Close(ctx)

const (
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/rowcontainer/hash_row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,7 +544,7 @@ func MakeHashDiskRowContainer(

// Init implements the HashRowContainer interface.
func (h *HashDiskRowContainer) Init(
_ context.Context, shouldMark bool, typs []*types.T, storedEqCols columns, encodeNull bool,
ctx context.Context, shouldMark bool, typs []*types.T, storedEqCols columns, encodeNull bool,
) error {
h.columnEncoder.init(typs, storedEqCols, encodeNull)
h.shouldMark = shouldMark
Expand All @@ -565,7 +565,7 @@ func (h *HashDiskRowContainer) Init(
}

var err error
h.DiskRowContainer, err = MakeDiskRowContainer(h.diskMonitor, storedTypes, storedEqColsToOrdering(storedEqCols), h.engine)
h.DiskRowContainer, err = MakeDiskRowContainer(ctx, h.diskMonitor, storedTypes, storedEqColsToOrdering(storedEqCols), h.engine)
return err
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/rowcontainer/kvstreamer_result_disk_buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ func (b *kvStreamerResultDiskBuffer) Serialize(
if !b.initialized {
var err error
b.container, err = MakeDiskRowContainer(
ctx,
b.monitor,
inOrderResultsBufferSpillTypeSchema,
colinfo.ColumnOrdering{},
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/rowcontainer/row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ func (f *DiskBackedRowContainer) SpillToDisk(ctx context.Context) error {
if f.UsingDisk() {
return errors.New("already using disk")
}
drc, err := MakeDiskRowContainer(f.diskMonitor, f.mrc.types, f.mrc.ordering, f.engine)
drc, err := MakeDiskRowContainer(ctx, f.diskMonitor, f.mrc.types, f.mrc.ordering, f.engine)
if err != nil {
return err
}
Expand Down

0 comments on commit 0d7d1ec

Please sign in to comment.