Skip to content

Commit

Permalink
roachpb: migrate BytesNext to encoding
Browse files Browse the repository at this point in the history
* BytesNext now lives in the encoding package.
* tree now uses encoding.BytesNext instead of roachpb.Key.Next

Release note: None
  • Loading branch information
otan committed Apr 28, 2022
1 parent 597a796 commit 80007da
Show file tree
Hide file tree
Showing 7 changed files with 29 additions and 32 deletions.
21 changes: 2 additions & 19 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func (rk RKey) Equal(other []byte) bool {
// The method may only take a shallow copy of the RKey, so both the
// receiver and the return value should be treated as immutable after.
func (rk RKey) Next() RKey {
return RKey(BytesNext(rk))
return RKey(encoding.BytesNext(rk))
}

// PrefixEnd determines the end key given key as a prefix, that is the
Expand All @@ -140,23 +140,6 @@ func (rk RKey) StringWithDirs(valDirs []encoding.Direction, maxLen int) string {
// messages which refer to Cockroach keys.
type Key []byte

// BytesNext returns the next possible byte slice, using the extra capacity
// of the provided slice if possible, and if not, appending an \x00.
func BytesNext(b []byte) []byte {
if cap(b) > len(b) {
bNext := b[:len(b)+1]
if bNext[len(bNext)-1] == 0 {
return bNext
}
}
// TODO(spencer): Do we need to enforce KeyMaxLength here?
// Switched to "make and copy" pattern in #4963 for performance.
bn := make([]byte, len(b)+1)
copy(bn, b)
bn[len(bn)-1] = 0
return bn
}

// Clone returns a copy of the key.
func (k Key) Clone() Key {
if k == nil {
Expand All @@ -171,7 +154,7 @@ func (k Key) Clone() Key {
// take a shallow copy of the Key, so both the receiver and the return
// value should be treated as immutable after.
func (k Key) Next() Key {
return Key(BytesNext(k))
return Key(encoding.BytesNext(k))
}

// IsPrev is a more efficient version of k.Next().Equal(m).
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/sem/tree/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,6 @@ go_library(
deps = [
"//pkg/geo",
"//pkg/geo/geopb",
"//pkg/roachpb",
"//pkg/security",
"//pkg/server/telemetry",
"//pkg/sql/catalog/catconstants",
Expand All @@ -143,6 +142,7 @@ go_library(
"//pkg/util/bitarray",
"//pkg/util/cache",
"//pkg/util/duration",
"//pkg/util/encoding",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/hlc",
"//pkg/util/ipaddr",
Expand Down Expand Up @@ -195,7 +195,6 @@ go_test(
"role_spec_test.go",
"table_name_test.go",
"time_test.go",
"timeconv_test.go",
"type_check_internal_test.go",
"type_check_test.go",
"type_name_test.go",
Expand All @@ -206,8 +205,6 @@ go_test(
deps = [
"//pkg/build/bazel",
"//pkg/internal/rsg",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/securitytest",
"//pkg/settings/cluster",
Expand All @@ -234,7 +231,6 @@ go_test(
"//pkg/util/log",
"//pkg/util/pretty",
"//pkg/util/randutil",
"//pkg/util/stop",
"//pkg/util/timeofday",
"//pkg/util/timetz",
"//pkg/util/timeutil",
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (

"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/geo"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand All @@ -36,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/bitarray"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/ipaddr"
"github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/stringencoding"
Expand Down Expand Up @@ -1260,7 +1260,7 @@ func (d *DString) Prev(ctx CompareContext) (Datum, bool) {

// Next implements the Datum interface.
func (d *DString) Next(ctx CompareContext) (Datum, bool) {
return NewDString(string(roachpb.Key(*d).Next())), true
return NewDString(string(encoding.BytesNext([]byte(*d)))), true
}

// IsMax implements the Datum interface.
Expand Down Expand Up @@ -1512,7 +1512,7 @@ func (d *DBytes) Prev(ctx CompareContext) (Datum, bool) {

// Next implements the Datum interface.
func (d *DBytes) Next(ctx CompareContext) (Datum, bool) {
return NewDBytes(DBytes(roachpb.Key(*d).Next())), true
return NewDBytes(DBytes(encoding.BytesNext([]byte(*d)))), true
}

// IsMax implements the Datum interface.
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,7 +394,7 @@ func BenchmarkScanAllIntentsResolved(b *testing.B) {
}
// Skip to next key.
buf = append(buf[:0], k.Key...)
buf = roachpb.BytesNext(buf)
buf = encoding.BytesNext(buf)
iter.SeekGE(MVCCKey{Key: buf})
}
}
Expand Down Expand Up @@ -444,7 +444,7 @@ func BenchmarkScanOneAllIntentsResolved(b *testing.B) {
}
// Skip to next key.
buf = append(buf[:0], k.Key...)
buf = roachpb.BytesNext(buf)
buf = encoding.BytesNext(buf)
iter.Close()
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4198,7 +4198,7 @@ func TestRandomizedMVCCResolveWriteIntentRange(t *testing.T) {
lu := roachpb.LockUpdate{
Span: roachpb.Span{
Key: puts[0].key,
EndKey: roachpb.BytesNext(puts[len(puts)-1].key),
EndKey: encoding.BytesNext(puts[len(puts)-1].key),
},
Txn: txnMeta,
Status: status,
Expand Down Expand Up @@ -4307,7 +4307,7 @@ func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) {
lu := roachpb.LockUpdate{
Span: roachpb.Span{
Key: puts[0].key,
EndKey: roachpb.BytesNext(puts[len(puts)-1].key),
EndKey: encoding.BytesNext(puts[len(puts)-1].key),
},
Txn: txn.TxnMeta,
Status: roachpb.PENDING,
Expand Down
3 changes: 2 additions & 1 deletion pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
Expand Down Expand Up @@ -1862,7 +1863,7 @@ func (p *pebbleReadOnly) rawMVCCGet(key []byte) ([]byte, error) {
}
options := pebble.IterOptions{
LowerBound: key,
UpperBound: roachpb.BytesNext(key),
UpperBound: encoding.BytesNext(key),
OnlyReadGuaranteedDurable: onlyReadGuaranteedDurable,
}
iter := p.parent.db.NewIter(&options)
Expand Down
17 changes: 17 additions & 0 deletions pkg/util/encoding/encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -3168,3 +3168,20 @@ func IsArrayKeyDone(buf []byte, dir Direction) bool {
}
return buf[0] == expected
}

// BytesNext returns the next possible byte slice, using the extra capacity
// of the provided slice if possible, and if not, appending an \x00.
func BytesNext(b []byte) []byte {
if cap(b) > len(b) {
bNext := b[:len(b)+1]
if bNext[len(bNext)-1] == 0 {
return bNext
}
}
// TODO(spencer): Do we need to enforce KeyMaxLength here?
// Switched to "make and copy" pattern in #4963 for performance.
bn := make([]byte, len(b)+1)
copy(bn, b)
bn[len(bn)-1] = 0
return bn
}

0 comments on commit 80007da

Please sign in to comment.