Skip to content

Commit

Permalink
rowcontainer: address an old TODO
Browse files Browse the repository at this point in the history
This commit addresses an old TODO about figuring out why we cannot reuse
the same buffer in `diskRowIterator.Row` method. The contract of that
method was previously confusing - it says that the call to `Row`
invalidates the row returned on the previous call; however, the
important piece was missing - that the datums themselves cannot be
mutated (this is what we assume elsewhere and perform only the "shallow"
copies). This commit clarifies the contract of the method and explicitly
explains why we need to allocate fresh byte slices (which is done via
`ByteAllocator` to reduce the number of allocations).

Additional context can be found in #43145 which added this copy in the
first place. Here is a relevant quote from Alfonso:
```
I think what's going on here is that this type of contract (you may only
reuse the row until the next call) is a bit unclear. `CopyRow`s of
`EncDatum`s are only implemented as shallow copies, so the reference to
this reuse only applies to the `EncDatumRow`, but not to the `encoded`
field, which is what is rewritten in this case. The `encoded` field will
not be copied, so I think the tests are probably failing due to that.
This is unfortunate and there doesn't seem to be a good reason for it.
To implement deep copies, we will have to implement deep copies for
`Datum`s as well.
```
I think we were under the impression that we'd implement the "deep copy"
in `CopyRow`, but I highly doubt we'll do so given that we're mostly
investing in the columnar infrastructure nowadays, and the current way
of performing shallow copying has worked well long enough.

Epic: None

Release note: None
  • Loading branch information
yuzefovich committed Dec 1, 2022
1 parent 80c4287 commit 170baed
Show file tree
Hide file tree
Showing 3 changed files with 15 additions and 18 deletions.
1 change: 1 addition & 0 deletions pkg/sql/rowcontainer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
29 changes: 12 additions & 17 deletions pkg/sql/rowcontainer/disk_row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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
}

Expand All @@ -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 {
Expand All @@ -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)
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/rowcontainer/row_container.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down

0 comments on commit 170baed

Please sign in to comment.