From ff890ab8e61bc44f6623315029601fd208bfdc1e Mon Sep 17 00:00:00 2001 From: Peter Mattis Date: Fri, 13 Dec 2019 09:12:34 -0500 Subject: [PATCH] storage/diskmap: remove SortedDiskMapIterator.{Key,Value} MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Remove `SortedDiskMapIterator.{Key,Value}` as these accessors are horribly slow due to performing an allocation on every call. Change the existing uses of these methods to `Unsafe{Key,Value}` adding copying when necessary. Most of the use cases were easy to fix, though note that `diskRowIterator.Row()` contains a TODO because the removal of the allocation there caused many test failures. The `PebbleMapIteration` benchmarks still show a regression in comparison to f5009c8a8b42fd4c5b073aaacd09f61f14f24e41. That regression is entirely due to Pebble's new memtable sizing heuristics which start with a small memtable size and dynamically grow the size up to the configured limit. Adding a knob to disable that behavior for the purposes of a benchmark does not seem worthwhile. name old time/op new time/op delta RocksDBMapIteration/InputSize4096-16 1.18ms ± 3% 0.81ms ± 0% -31.56% (p=0.000 n=10+8) RocksDBMapIteration/InputSize16384-16 5.83ms ± 1% 4.14ms ± 3% -29.13% (p=0.000 n=9+10) RocksDBMapIteration/InputSize65536-16 24.8ms ± 1% 17.7ms ± 3% -28.54% (p=0.000 n=9+10) RocksDBMapIteration/InputSize262144-16 137ms ± 0% 105ms ± 2% -23.65% (p=0.000 n=9+9) RocksDBMapIteration/InputSize1048576-16 547ms ± 1% 430ms ± 1% -21.44% (p=0.000 n=8+9) PebbleMapIteration/InputSize4096-16 594µs ± 1% 323µs ± 2% -45.65% (p=0.000 n=9+9) PebbleMapIteration/InputSize16384-16 3.29ms ± 1% 2.15ms ± 1% -34.70% (p=0.000 n=10+9) PebbleMapIteration/InputSize65536-16 16.0ms ± 7% 11.2ms ±11% -30.26% (p=0.000 n=10+10) PebbleMapIteration/InputSize262144-16 96.7ms ± 3% 76.5ms ± 5% -20.88% (p=0.000 n=10+10) PebbleMapIteration/InputSize1048576-16 267ms ± 0% 185ms ± 1% -30.60% (p=0.000 n=9+10) name old alloc/op new alloc/op delta RocksDBMapIteration/InputSize4096-16 262kB ± 0% 0kB ± 0% -99.97% (p=0.000 n=10+10) RocksDBMapIteration/InputSize16384-16 1.31MB ± 0% 0.00MB ± 0% -99.99% (p=0.000 n=10+10) RocksDBMapIteration/InputSize65536-16 5.51MB ± 0% 0.00MB ± 3% -100.00% (p=0.000 n=10+10) RocksDBMapIteration/InputSize262144-16 22.3MB ± 0% 0.0MB ± 0% -100.00% (p=0.000 n=10+10) RocksDBMapIteration/InputSize1048576-16 89.4MB ± 0% 0.0MB ± 0% -100.00% (p=0.000 n=10+9) PebbleMapIteration/InputSize4096-16 263kB ± 0% 0kB ± 0% -99.91% (p=0.000 n=10+10) PebbleMapIteration/InputSize16384-16 1.31MB ± 0% 0.00MB ± 0% -99.98% (p=0.000 n=10+8) PebbleMapIteration/InputSize65536-16 5.50MB ± 0% 0.00MB ± 3% -99.99% (p=0.000 n=10+9) PebbleMapIteration/InputSize262144-16 22.3MB ± 0% 0.0MB ± 0% -99.99% (p=0.000 n=10+7) PebbleMapIteration/InputSize1048576-16 89.3MB ± 0% 0.0MB ±26% -100.00% (p=0.000 n=10+10) name old allocs/op new allocs/op delta RocksDBMapIteration/InputSize4096-16 8.20k ± 0% 0.00k ± 0% -99.96% (p=0.000 n=10+10) RocksDBMapIteration/InputSize16384-16 41.0k ± 0% 0.0k ± 0% -99.99% (p=0.000 n=10+10) RocksDBMapIteration/InputSize65536-16 172k ± 0% 0k ± 0% -100.00% (p=0.000 n=10+10) RocksDBMapIteration/InputSize262144-16 696k ± 0% 0k ± 0% -100.00% (p=0.000 n=10+10) RocksDBMapIteration/InputSize1048576-16 2.79M ± 0% 0.00M ± 0% -100.00% (p=0.000 n=9+9) PebbleMapIteration/InputSize4096-16 8.20k ± 0% 0.01k ± 0% -99.94% (p=0.000 n=10+10) PebbleMapIteration/InputSize16384-16 41.0k ± 0% 0.0k ± 0% -99.99% (p=0.000 n=10+10) PebbleMapIteration/InputSize65536-16 172k ± 0% 0k ± 0% -100.00% (p=0.000 n=10+10) PebbleMapIteration/InputSize262144-16 696k ± 0% 0k ± 0% -100.00% (p=0.000 n=10+10) PebbleMapIteration/InputSize1048576-16 2.79M ± 0% 0.00M ± 9% -100.00% (p=0.000 n=10+10) Release note: None --- pkg/sql/rowcontainer/disk_row_container.go | 22 +++++++++++++++-- pkg/sql/rowcontainer/hash_row_container.go | 16 ++++++------- pkg/storage/diskmap/disk_map.go | 8 +------ pkg/storage/engine/disk_map.go | 28 ---------------------- pkg/storage/engine/disk_map_test.go | 10 ++++---- 5 files changed, 34 insertions(+), 50 deletions(-) diff --git a/pkg/sql/rowcontainer/disk_row_container.go b/pkg/sql/rowcontainer/disk_row_container.go index fec633921c57..1e1b632d7c28 100644 --- a/pkg/sql/rowcontainer/disk_row_container.go +++ b/pkg/sql/rowcontainer/disk_row_container.go @@ -275,6 +275,7 @@ func (d *DiskRowContainer) keyValToRow(k []byte, v []byte) (sqlbase.EncDatumRow, // diskRowIterator iterates over the rows in a DiskRowContainer. type diskRowIterator struct { rowContainer *DiskRowContainer + rowBuf []byte diskmap.SortedDiskMapIterator } @@ -305,7 +306,23 @@ func (r *diskRowIterator) Row() (sqlbase.EncDatumRow, error) { return nil, errors.AssertionFailedf("invalid row") } - return r.rowContainer.keyValToRow(r.Key(), r.Value()) + 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):] + + return r.rowContainer.keyValToRow(k, v) } func (r *diskRowIterator) Close() { @@ -347,7 +364,8 @@ func (r *diskRowFinalIterator) Row() (sqlbase.EncDatumRow, error) { if err != nil { return nil, err } - r.diskRowIterator.rowContainer.lastReadKey = r.Key() + r.diskRowIterator.rowContainer.lastReadKey = + append(r.diskRowIterator.rowContainer.lastReadKey[:0], r.UnsafeKey()...) return row, nil } diff --git a/pkg/sql/rowcontainer/hash_row_container.go b/pkg/sql/rowcontainer/hash_row_container.go index efb9d1a87fc2..7e98ad86fc0f 100644 --- a/pkg/sql/rowcontainer/hash_row_container.go +++ b/pkg/sql/rowcontainer/hash_row_container.go @@ -553,6 +553,8 @@ type hashDiskRowBucketIterator struct { // encodedEqCols is the encoding of the equality columns of the rows in the // bucket that this iterator iterates over. encodedEqCols []byte + // Temporary buffer used for constructed marked values. + tmpBuf []byte } var _ RowMarkerIterator = &hashDiskRowBucketIterator{} @@ -586,10 +588,7 @@ func (i *hashDiskRowBucketIterator) Valid() (bool, error) { } // Since the underlying map is sorted, once the key prefix does not equal // the encoded equality columns, we have gone past the end of the bucket. - // TODO(asubiotto): Make UnsafeKey() and UnsafeValue() part of the - // SortedDiskMapIterator interface to avoid allocation here, in Mark(), and - // isRowMarked(). - return bytes.HasPrefix(i.Key(), i.encodedEqCols), nil + return bytes.HasPrefix(i.UnsafeKey(), i.encodedEqCols), nil } // Row implements the RowIterator interface. @@ -632,7 +631,7 @@ func (i *hashDiskRowBucketIterator) IsMarked(ctx context.Context) bool { return false } - rowVal := i.Value() + rowVal := i.UnsafeValue() return bytes.Equal(rowVal[len(rowVal)-len(encodedTrue):], encodedTrue) } @@ -648,7 +647,7 @@ func (i *hashDiskRowBucketIterator) Mark(ctx context.Context, mark bool) error { } // rowVal are the non-equality encoded columns, the last of which is the // column we use to mark a row. - rowVal := i.Value() + rowVal := append(i.tmpBuf[:0], i.UnsafeValue()...) originalLen := len(rowVal) rowVal = append(rowVal, markBytes...) @@ -656,11 +655,12 @@ func (i *hashDiskRowBucketIterator) Mark(ctx context.Context, mark bool) error { // the extra bytes. copy(rowVal[originalLen-len(markBytes):], rowVal[originalLen:]) rowVal = rowVal[:originalLen] + i.tmpBuf = rowVal // These marks only matter when using a hashDiskRowIterator to iterate over // unmarked rows. The writes are flushed when creating a NewIterator() in // NewUnmarkedIterator(). - return i.HashDiskRowContainer.bufferedRows.Put(i.Key(), rowVal) + return i.HashDiskRowContainer.bufferedRows.Put(i.UnsafeKey(), rowVal) } // hashDiskRowIterator iterates over all unmarked rows in a @@ -720,7 +720,7 @@ func (i *hashDiskRowIterator) isRowMarked() bool { return false } - rowVal := i.Value() + rowVal := i.UnsafeValue() return bytes.Equal(rowVal[len(rowVal)-len(encodedTrue):], encodedTrue) } diff --git a/pkg/storage/diskmap/disk_map.go b/pkg/storage/diskmap/disk_map.go index cb69475f1a8e..a398fab6d1f0 100644 --- a/pkg/storage/diskmap/disk_map.go +++ b/pkg/storage/diskmap/disk_map.go @@ -33,7 +33,7 @@ type Factory interface { // } else if !ok { // break // } -// key := i.Key() +// key := i.UnsafeKey() // // Do something. // } type SortedDiskMapIterator interface { @@ -49,12 +49,6 @@ type SortedDiskMapIterator interface { Valid() (bool, error) // Next advances the iterator to the next key in the iteration. Next() - // Key returns the current key. The resulting byte slice is still valid - // after the next call to Seek(), Rewind(), or Next(). - Key() []byte - // Value returns the current value. The resulting byte slice is still valid - // after the next call to Seek(), Rewind(), or Next(). - Value() []byte // UnsafeKey returns the same value as Key, but the memory is invalidated on // the next call to {Next,Rewind,Seek,Close}. diff --git a/pkg/storage/engine/disk_map.go b/pkg/storage/engine/disk_map.go index 242cccaef535..670b78a8b661 100644 --- a/pkg/storage/engine/disk_map.go +++ b/pkg/storage/engine/disk_map.go @@ -194,16 +194,6 @@ func (i *rocksDBMapIterator) Next() { i.iter.Next() } -// Key implements the SortedDiskMapIterator interface. -func (i *rocksDBMapIterator) Key() []byte { - return i.iter.Key().Key[len(i.prefix):] -} - -// Value implements the SortedDiskMapIterator interface. -func (i *rocksDBMapIterator) Value() []byte { - return i.iter.Value() -} - // UnsafeKey implements the SortedDiskMapIterator interface. func (i *rocksDBMapIterator) UnsafeKey() []byte { return i.iter.UnsafeKey().Key[len(i.prefix):] @@ -396,24 +386,6 @@ func (i *pebbleMapIterator) Next() { i.iter.Next() } -// Key implements the SortedDiskMapIterator interface. -func (i *pebbleMapIterator) Key() []byte { - unsafeKey := i.UnsafeKey() - safeKey := make([]byte, len(unsafeKey)) - copy(safeKey, unsafeKey) - - return safeKey -} - -// Value implements the SortedDiskMapIterator interface. -func (i *pebbleMapIterator) Value() []byte { - unsafeValue := i.iter.Value() - safeValue := make([]byte, len(unsafeValue)) - copy(safeValue, unsafeValue) - - return safeValue -} - // UnsafeKey implements the SortedDiskMapIterator interface. func (i *pebbleMapIterator) UnsafeKey() []byte { unsafeKey := i.iter.Key() diff --git a/pkg/storage/engine/disk_map_test.go b/pkg/storage/engine/disk_map_test.go index 5c881ef698aa..2cdf3bc8ba65 100644 --- a/pkg/storage/engine/disk_map_test.go +++ b/pkg/storage/engine/disk_map_test.go @@ -171,7 +171,7 @@ func runTestForEngine(ctx context.Context, t *testing.T, filename string, engine } valid, err := iter.Valid() if valid && err == nil { - fmt.Fprintf(&b, "%s:%s\n", iter.Key(), iter.Value()) + fmt.Fprintf(&b, "%s:%s\n", iter.UnsafeKey(), iter.UnsafeValue()) } else if err != nil { fmt.Fprintf(&b, "err=%v\n", err) } else { @@ -381,8 +381,8 @@ func BenchmarkRocksDBMapIteration(b *testing.B) { } else if !ok { break } - i.Key() - i.Value() + i.UnsafeKey() + i.UnsafeValue() } i.Close() } @@ -521,8 +521,8 @@ func BenchmarkPebbleMapIteration(b *testing.B) { } else if !ok { break } - i.Key() - i.Value() + i.UnsafeKey() + i.UnsafeValue() } i.Close() }