diff --git a/pkg/sql/rowcontainer/BUILD.bazel b/pkg/sql/rowcontainer/BUILD.bazel index 44db8cc14039..90d87cf8de36 100644 --- a/pkg/sql/rowcontainer/BUILD.bazel +++ b/pkg/sql/rowcontainer/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/sql/sqlerrors", "//pkg/sql/types", "//pkg/util", + "//pkg/util/bufalloc", "//pkg/util/cancelchecker", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/sql/rowcontainer/disk_row_container.go b/pkg/sql/rowcontainer/disk_row_container.go index d79479501806..88cc9aad7524 100644 --- a/pkg/sql/rowcontainer/disk_row_container.go +++ b/pkg/sql/rowcontainer/disk_row_container.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/bufalloc" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -394,8 +395,9 @@ func (d *DiskRowContainer) Close(ctx context.Context) { } // keyValToRow decodes a key and a value byte slice stored with AddRow() into -// a sqlbase.EncDatumRow. The returned EncDatumRow is only valid until the next -// call to keyValToRow(). +// a rowenc.EncDatumRow. The returned EncDatumRow is only valid until the next +// call to keyValToRow(). The passed in byte slices are used directly, and it is +// the caller's responsibility to make sure they don't get modified. func (d *DiskRowContainer) keyValToRow(k []byte, v []byte) (rowenc.EncDatumRow, error) { for i, orderInfo := range d.ordering { // Types with composite key encodings are decoded from the value. @@ -430,7 +432,7 @@ func (d *DiskRowContainer) keyValToRow(k []byte, v []byte) (rowenc.EncDatumRow, // diskRowIterator iterates over the rows in a DiskRowContainer. type diskRowIterator struct { rowContainer *DiskRowContainer - rowBuf []byte + rowBuf bufalloc.ByteAllocator diskmap.SortedDiskMapIterator } @@ -452,7 +454,7 @@ func (d *DiskRowContainer) NewIterator(ctx context.Context) RowIterator { return &i } -// Row returns the current row. The returned sqlbase.EncDatumRow is only valid +// Row returns the current row. The returned rowenc.EncDatumRow is only valid // until the next call to Row(). func (r *diskRowIterator) Row() (rowenc.EncDatumRow, error) { if ok, err := r.Valid(); err != nil { @@ -463,19 +465,12 @@ func (r *diskRowIterator) Row() (rowenc.EncDatumRow, error) { k := r.UnsafeKey() v := r.UnsafeValue() - // TODO(asubiotto): the "true ||" should not be necessary. We should be to - // reuse rowBuf, yet doing so causes - // TestDiskBackedIndexedRowContainer/ReorderingOnDisk, TestHashJoiner, and - // TestSorter to fail. Some caller of Row() is presumably not making a copy - // of the return value. - if true || cap(r.rowBuf) < len(k)+len(v) { - r.rowBuf = make([]byte, 0, len(k)+len(v)) - } - r.rowBuf = r.rowBuf[:len(k)+len(v)] - copy(r.rowBuf, k) - copy(r.rowBuf[len(k):], v) - k = r.rowBuf[:len(k)] - v = r.rowBuf[len(k):] + // keyValToRow will use the encoded key and value bytes as is by shoving + // them directly into the EncDatum, so we need to make a copy here. We + // cannot reuse the same byte slice across Row() calls because it would lead + // to modification of the EncDatums (which is not allowed). + r.rowBuf, k = r.rowBuf.Copy(k, len(v)) + r.rowBuf, v = r.rowBuf.Copy(v, 0 /* extraCap */) return r.rowContainer.keyValToRow(k, v) } diff --git a/pkg/sql/rowcontainer/row_container.go b/pkg/sql/rowcontainer/row_container.go index ba59fa6c021e..fcdb75d6b50d 100644 --- a/pkg/sql/rowcontainer/row_container.go +++ b/pkg/sql/rowcontainer/row_container.go @@ -132,7 +132,8 @@ type RowIterator interface { // Next advances the iterator to the next row in the iteration. Next() // Row returns the current row. The returned row is only valid until the - // next call to Rewind() or Next(). + // next call to Rewind() or Next(). However, datums in the row won't be + // modified, so shallow copying is sufficient. Row() (rowenc.EncDatumRow, error) // Close frees up resources held by the iterator.