From db6c8b4f121d6ff7a715f4037001abe625998e92 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 18:19:11 -0400 Subject: [PATCH 01/12] bench: add a large insert benchmark MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This benchmark shows the reducation in allocations by the copy avoidance changes. Together the changes result in: ``` name old time/op new time/op delta SQL/Cockroach/InsertLarge/count=1000-10 18.7ms ± 4% 19.1ms ±26% ~ (p=0.780 n=9+10) name old alloc/op new alloc/op delta SQL/Cockroach/InsertLarge/count=1000-10 49.7MB ± 4% 39.7MB ±10% -20.19% (p=0.000 n=9+9) name old allocs/op new allocs/op delta SQL/Cockroach/InsertLarge/count=1000-10 39.0k ± 5% 29.7k ± 8% -23.83% (p=0.000 n=10+9) ``` Release note: None Epic: None --- pkg/bench/bench_test.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/pkg/bench/bench_test.go b/pkg/bench/bench_test.go index 37885b7e4ff1..e51bc84db62e 100644 --- a/pkg/bench/bench_test.go +++ b/pkg/bench/bench_test.go @@ -304,6 +304,36 @@ func runBenchmarkInsert(b *testing.B, db *sqlutils.SQLRunner, count int) { } +// runBenchmarkInsertLarge benchmarks inserting count large rows into a table +// where large means rows with a 1k string and 1k BYTES object. +func runBenchmarkInsertLarge(b *testing.B, db *sqlutils.SQLRunner, count int) { + defer func() { + db.Exec(b, `DROP TABLE IF EXISTS bench.insert`) + }() + + db.Exec(b, `CREATE TABLE bench.insert (k INT PRIMARY KEY, s STRING, b BYTES)`) + bigstr := strings.Repeat("x", 1<<10) + bigbytes := bytes.Repeat([]byte("x"), 1<<10) + + b.ResetTimer() + var buf bytes.Buffer + val := 0 + for i := 0; i < b.N; i++ { + buf.Reset() + buf.WriteString(`INSERT INTO bench.insert VALUES `) + for j := 0; j < count; j++ { + if j > 0 { + buf.WriteString(", ") + } + fmt.Fprintf(&buf, "(%d, '%s', '%s')", val, bigstr, bigbytes) + val++ + } + db.Exec(b, buf.String()) + } + b.StopTimer() + +} + // runBenchmarkInsertFK benchmarks inserting count rows into a table with a // present foreign key into another table. func runBenchmarkInsertFK(b *testing.B, db *sqlutils.SQLRunner, count int) { @@ -419,6 +449,7 @@ func BenchmarkSQL(b *testing.B) { for _, runFn := range []func(*testing.B, *sqlutils.SQLRunner, int){ runBenchmarkDelete, runBenchmarkInsert, + runBenchmarkInsertLarge, runBenchmarkInsertDistinct, runBenchmarkInsertFK, runBenchmarkInsertSecondaryIndex, From 2bf2323e7bc502b21d561985a62b34f65d08bdd2 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 18:16:15 -0400 Subject: [PATCH 02/12] lex: avoid some string to byte slice copies Facilitate some copy avoidance by using []byte instead of string. Only copy when necessary in some cases. Release note: None Epic: None --- pkg/sql/lex/encode.go | 29 +++++++++++++++++++++-------- pkg/sql/lex/encode_test.go | 4 ++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go index e02b9effdd93..c5b9163ed2bb 100644 --- a/pkg/sql/lex/encode.go +++ b/pkg/sql/lex/encode.go @@ -142,28 +142,39 @@ func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix // according to the encoding specification in "be". // When using the Hex format, the caller is responsible for skipping the // "\x" prefix, if any. See DecodeRawBytesToByteArrayAuto() below for -// an alternative. -func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error) { +// an alternative. If no conversion is necessary the input is returned, +// callers should not assume a copy is made. +func DecodeRawBytesToByteArray(data []byte, be BytesEncodeFormat) ([]byte, error) { switch be { case BytesEncodeHex: - return hex.DecodeString(data) + res := make([]byte, hex.DecodedLen(len(data))) + n, err := hex.Decode(res, data) + return res[:n], err case BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. // See https://www.postgresql.org/docs/current/static/datatype-binary.html#AEN5667 - res := make([]byte, 0, len(data)) + res := data + copied := false for i := 0; i < len(data); i++ { ch := data[i] if ch != '\\' { - res = append(res, ch) + if copied { + res = append(res, ch) + } continue } if i >= len(data)-1 { return nil, pgerror.New(pgcode.InvalidEscapeSequence, "bytea encoded value ends with escape character") } + if !copied { + res = make([]byte, 0, len(data)) + res = append(res, data[:i]...) + copied = true + } if data[i+1] == '\\' { res = append(res, '\\') i++ @@ -188,7 +199,9 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error return res, nil case BytesEncodeBase64: - return base64.StdEncoding.DecodeString(data) + res := make([]byte, base64.StdEncoding.DecodedLen(len(data))) + n, err := base64.StdEncoding.Decode(res, data) + return res[:n], err default: return nil, errors.AssertionFailedf("unhandled format: %s", be) @@ -200,9 +213,9 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error // and escape. func DecodeRawBytesToByteArrayAuto(data []byte) ([]byte, error) { if len(data) >= 2 && data[0] == '\\' && (data[1] == 'x' || data[1] == 'X') { - return DecodeRawBytesToByteArray(string(data[2:]), BytesEncodeHex) + return DecodeRawBytesToByteArray(data[2:], BytesEncodeHex) } - return DecodeRawBytesToByteArray(string(data), BytesEncodeEscape) + return DecodeRawBytesToByteArray(data, BytesEncodeEscape) } func (f BytesEncodeFormat) String() string { diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go index f36bfb82e683..9cd63c864443 100644 --- a/pkg/sql/lex/encode_test.go +++ b/pkg/sql/lex/encode_test.go @@ -64,7 +64,7 @@ func TestByteArrayDecoding(t *testing.T) { if s.auto { dec, err = lex.DecodeRawBytesToByteArrayAuto([]byte(s.in)) } else { - dec, err = lex.DecodeRawBytesToByteArray(s.in, s.inFmt) + dec, err = lex.DecodeRawBytesToByteArray([]byte(s.in), s.inFmt) } if s.err != "" { if err == nil { @@ -124,7 +124,7 @@ func TestByteArrayEncoding(t *testing.T) { enc = enc[2:] } - dec, err := lex.DecodeRawBytesToByteArray(enc, format) + dec, err := lex.DecodeRawBytesToByteArray([]byte(enc), format) if err != nil { t.Fatal(err) } From b338d7bab0c62c061068a8b89b5205a951c04f96 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 15:06:52 -0400 Subject: [PATCH 03/12] encoding: unsafeString -> UnsafeConvertBytesToString Make public for use elsewhere. Release note: None Epic: None --- pkg/util/encoding/encoding.go | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index 22f573d86edf..c80114c3216e 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -843,7 +843,7 @@ func prettyPrintInvertedIndexKey(b []byte) (string, []byte, error) { switch tempB[i+1] { case escapedTerm: if len(tempB[:i]) > 0 { - outBytes = outBytes + strconv.Quote(unsafeString(tempB[:i])) + outBytes = outBytes + strconv.Quote(UnsafeConvertBytesToString(tempB[:i])) } else { lenOut := len(outBytes) if lenOut > 1 && outBytes[lenOut-1] == '/' { @@ -852,7 +852,7 @@ func prettyPrintInvertedIndexKey(b []byte) (string, []byte, error) { } return outBytes, tempB[i+escapeLength:], nil case escapedJSONObjectKeyTerm: - outBytes = outBytes + strconv.Quote(unsafeString(tempB[:i])) + "/" + outBytes = outBytes + strconv.Quote(UnsafeConvertBytesToString(tempB[:i])) + "/" case escapedJSONArray: outBytes = outBytes + "Arr/" default: @@ -956,11 +956,12 @@ func EncodeStringDescending(b []byte, s string) []byte { return EncodeBytesDescending(b, arg) } -// unsafeString performs an unsafe conversion from a []byte to a string. The -// returned string will share the underlying memory with the []byte which thus -// allows the string to be mutable through the []byte. We're careful to use -// this method only in situations in which the []byte will not be modified. -func unsafeString(b []byte) string { +// UnsafeConvertBytesToString performs an unsafe conversion from a []byte to a +// string. The returned string will share the underlying memory with the +// []byte which thus allows the string to be mutable through the []byte. We're +// careful to use this method only in situations in which the []byte will not +// be modified. +func UnsafeConvertBytesToString(b []byte) string { return *(*string)(unsafe.Pointer(&b)) } @@ -971,7 +972,7 @@ func unsafeString(b []byte) string { // string may share storage with the input buffer. func DecodeUnsafeStringAscending(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesAscending(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // DecodeUnsafeStringAscendingDeepCopy is the same as @@ -979,7 +980,7 @@ func DecodeUnsafeStringAscending(b []byte, r []byte) ([]byte, string, error) { // with the input buffer. func DecodeUnsafeStringAscendingDeepCopy(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesAscendingDeepCopy(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // DecodeUnsafeStringDescending decodes a string value from the input buffer which @@ -990,7 +991,7 @@ func DecodeUnsafeStringAscendingDeepCopy(b []byte, r []byte) ([]byte, string, er // buffer. func DecodeUnsafeStringDescending(b []byte, r []byte) ([]byte, string, error) { b, r, err := DecodeBytesDescending(b, r) - return b, unsafeString(r), err + return b, UnsafeConvertBytesToString(r), err } // EncodeNullAscending encodes a NULL value. The encodes bytes are appended to the From 2d83a737077d69ab4562195f807c13641312f027 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 14:45:19 -0400 Subject: [PATCH 04/12] tree: allow access to DBytes/DString/DEncodedKey raw bytes Epic: None Release note: None --- pkg/sql/sem/tree/datum.go | 32 ++++++++++++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 4ab3a74dab5f..6dc96ed1fc55 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -374,11 +374,11 @@ func ParseDBool(s string) (*DBool, error) { // the beginning), and the escaped format, which supports "\\" and // octal escapes. func ParseDByte(s string) (*DBytes, error) { - res, err := lex.DecodeRawBytesToByteArrayAuto([]byte(s)) + res, err := lex.DecodeRawBytesToByteArrayAuto(encoding.UnsafeConvertStringToBytes(s)) if err != nil { return nil, MakeParseError(s, types.Bytes, err) } - return NewDBytes(DBytes(res)), nil + return NewDBytes(DBytes(encoding.UnsafeConvertBytesToString(res))), nil } // ParseDUuidFromString parses and returns the *DUuid Datum value represented @@ -1345,6 +1345,13 @@ func (d *DString) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "Unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DString) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DCollatedString is the Datum for strings with a locale. The struct members // are intended to be immutable. type DCollatedString struct { @@ -1485,6 +1492,13 @@ func (d *DCollatedString) IsComposite() bool { return true } +// UnsafeContentBytes returns the raw bytes avoiding allocation. It is "unsafe" +// because the contract is that callers must not to mutate the bytes but there +// is nothing stopping that from happening. +func (d *DCollatedString) UnsafeContentBytes() []byte { + return encoding.UnsafeConvertStringToBytes(d.Contents) +} + // DBytes is the bytes Datum. The underlying type is a string because we want // the immutability, but this may contain arbitrary bytes. type DBytes string @@ -1617,6 +1631,13 @@ func (d *DBytes) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DBytes) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DEncodedKey is a special Datum of types.EncodedKey type, used to pass through // encoded key data. It is similar to DBytes, except when it comes to // encoding/decoding. It is currently used to pass around inverted index keys, @@ -1689,6 +1710,13 @@ func (d *DEncodedKey) Size() uintptr { return unsafe.Sizeof(*d) + uintptr(len(*d)) } +// UnsafeBytes returns the raw bytes avoiding allocation. It is "unsafe" because +// the contract is that callers must not to mutate the bytes but there is +// nothing stopping that from happening. +func (d *DEncodedKey) UnsafeBytes() []byte { + return encoding.UnsafeConvertStringToBytes(string(*d)) +} + // DUuid is the UUID Datum. type DUuid struct { uuid.UUID From 2f556fc64ccd5c3c250e1028348872782376b403 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 14:46:01 -0400 Subject: [PATCH 05/12] opt: avoid allocation in DBytes interning DBytes is a string under the covers but we usually operate on them with []byte APIs, avoid copies in these cases. Release note: None Epic: None --- pkg/sql/opt/memo/interner.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sql/opt/memo/interner.go b/pkg/sql/opt/memo/interner.go index 2e0f86c470d1..db43fb6748d9 100644 --- a/pkg/sql/opt/memo/interner.go +++ b/pkg/sql/opt/memo/interner.go @@ -361,7 +361,7 @@ func (h *hasher) HashDatum(val tree.Datum) { case *tree.DString: h.HashString(string(*t)) case *tree.DBytes: - h.HashBytes([]byte(*t)) + h.HashBytes(t.UnsafeBytes()) case *tree.DDate: h.HashUint64(uint64(t.PGEpochDays())) case *tree.DTime: @@ -843,7 +843,7 @@ func (h *hasher) IsDatumEqual(l, r tree.Datum) bool { return lt.Locale == rt.Locale && h.IsStringEqual(lt.Contents, rt.Contents) case *tree.DBytes: rt := r.(*tree.DBytes) - return bytes.Equal([]byte(*lt), []byte(*rt)) + return bytes.Equal(lt.UnsafeBytes(), rt.UnsafeBytes()) case *tree.DDate: rt := r.(*tree.DDate) return lt.Date == rt.Date From 3f225e415b77e36237e2e36dd3d28d9d56962936 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 15:08:57 -0400 Subject: [PATCH 06/12] colenc: use new UnsafeConvertBytesToString Cosmetic change to use the new UnsafeConvertBytesToString. Release note: None Epic: None --- pkg/sql/colenc/inverted.go | 3 ++- pkg/sql/colenc/key.go | 10 ++-------- pkg/sql/colenc/legacy.go | 3 ++- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/pkg/sql/colenc/inverted.go b/pkg/sql/colenc/inverted.go index ce1cbec7ffe1..238ffedf0166 100644 --- a/pkg/sql/colenc/inverted.go +++ b/pkg/sql/colenc/inverted.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) @@ -32,7 +33,7 @@ func invertedColToDatum(vec coldata.Vec, row int) tree.Datum { return tree.NewDJSON(vec.JSON().Get(row)) case types.StringFamily: b := vec.Bytes().Get(row) - s := unsafeConvertBytesToString(b) + s := encoding.UnsafeConvertBytesToString(b) return tree.NewDString(s) } // This handles arrays, geo etc. diff --git a/pkg/sql/colenc/key.go b/pkg/sql/colenc/key.go index 598cc708d051..0fe6a062a6b9 100644 --- a/pkg/sql/colenc/key.go +++ b/pkg/sql/colenc/key.go @@ -11,8 +11,6 @@ package colenc import ( - "unsafe" - "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/typeconv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -144,9 +142,9 @@ func encodeKeys[T []byte | roachpb.Key]( } s := ss.Get(r + start) if dir == encoding.Ascending { - kys[r] = encoding.EncodeStringAscending(b, unsafeConvertBytesToString(s)) + kys[r] = encoding.EncodeStringAscending(b, encoding.UnsafeConvertBytesToString(s)) } else { - kys[r] = encoding.EncodeStringDescending(b, unsafeConvertBytesToString(s)) + kys[r] = encoding.EncodeStringDescending(b, encoding.UnsafeConvertBytesToString(s)) } } case types.TimestampFamily, types.TimestampTZFamily: @@ -249,7 +247,3 @@ func (b *BatchEncoder) encodeIndexKey( } return nil } - -func unsafeConvertBytesToString(b []byte) string { - return *(*string)(unsafe.Pointer(&b)) -} diff --git a/pkg/sql/colenc/legacy.go b/pkg/sql/colenc/legacy.go index 5e15dffa19c9..fafff4b0ca54 100644 --- a/pkg/sql/colenc/legacy.go +++ b/pkg/sql/colenc/legacy.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc/valueside" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -66,7 +67,7 @@ func MarshalLegacy(colType *types.T, vec coldata.Vec, row int) (roachpb.Value, e switch vec.Type().Family() { case types.StringFamily, types.BytesFamily, types.UuidFamily, types.EnumFamily: b := vec.Bytes().Get(row) - r.SetString(unsafeConvertBytesToString(b)) + r.SetString(encoding.UnsafeConvertBytesToString(b)) return r, nil } case types.TimestampFamily, types.TimestampTZFamily: From e76872bffec817ac231745b3e6ce6590f510881f Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 15:19:58 -0400 Subject: [PATCH 07/12] valueside: remove string to []byte copies Release note: None Epic: None --- pkg/sql/rowenc/valueside/encode.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/sql/rowenc/valueside/encode.go b/pkg/sql/rowenc/valueside/encode.go index 33b548b42bea..506cd0911efc 100644 --- a/pkg/sql/rowenc/valueside/encode.go +++ b/pkg/sql/rowenc/valueside/encode.go @@ -48,11 +48,11 @@ func Encode(appendTo []byte, colID ColumnIDDelta, val tree.Datum, scratch []byte case *tree.DDecimal: return encoding.EncodeDecimalValue(appendTo, uint32(colID), &t.Decimal), nil case *tree.DString: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DBytes: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DEncodedKey: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(*t)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeBytes()), nil case *tree.DDate: return encoding.EncodeIntValue(appendTo, uint32(colID), t.UnixEpochDaysWithOrig()), nil case *tree.DBox2D: @@ -102,7 +102,7 @@ func Encode(appendTo []byte, colID ColumnIDDelta, val tree.Datum, scratch []byte case *tree.DTuple: return encodeTuple(t, appendTo, uint32(colID), scratch) case *tree.DCollatedString: - return encoding.EncodeBytesValue(appendTo, uint32(colID), []byte(t.Contents)), nil + return encoding.EncodeBytesValue(appendTo, uint32(colID), t.UnsafeContentBytes()), nil case *tree.DOid: return encoding.EncodeIntValue(appendTo, uint32(colID), int64(t.Oid)), nil case *tree.DEnum: From 81fec5a0a20a672002b47e78d01ca55f368c4ddb Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 18:18:03 -0400 Subject: [PATCH 08/12] builtins: avoid some []byte to string copies in decode Release note: None Epic: None --- pkg/sql/sem/builtins/builtins.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index b45d1d336565..eab59a5f97f5 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -1164,11 +1164,11 @@ var regularBuiltins = map[string]builtinDefinition{ return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for decode()") } - res, err := lex.DecodeRawBytesToByteArray(data, be) + res, err := lex.DecodeRawBytesToByteArray(encoding.UnsafeConvertStringToBytes(data), be) if err != nil { return nil, err } - return tree.NewDBytes(tree.DBytes(res)), nil + return tree.NewDBytes(tree.DBytes(encoding.UnsafeConvertBytesToString(res))), nil }, Info: "Decodes `data` using `format` (`hex` / `escape` / `base64`).", Volatility: volatility.Immutable, From 83870fcdac80886ea23af4da998899ac952ec545 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 17 Apr 2023 18:18:28 -0400 Subject: [PATCH 09/12] pgwire: avoid some string to byte copies Most parse routines don't retain the input string pointer so we can use a pointer to the input bytes in those cases. Release note: None Epic: None --- pkg/sql/pgwire/pgwirebase/BUILD.bazel | 1 + pkg/sql/pgwire/pgwirebase/encoding.go | 102 +++++++++++++++----------- 2 files changed, 59 insertions(+), 44 deletions(-) diff --git a/pkg/sql/pgwire/pgwirebase/BUILD.bazel b/pkg/sql/pgwire/pgwirebase/BUILD.bazel index 489e4f28f33c..c44bee349eb0 100644 --- a/pkg/sql/pgwire/pgwirebase/BUILD.bazel +++ b/pkg/sql/pgwire/pgwirebase/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/sql/types", "//pkg/util/bitarray", "//pkg/util/duration", + "//pkg/util/encoding", "//pkg/util/errorutil/unimplemented", "//pkg/util/ipaddr", "//pkg/util/timeofday", diff --git a/pkg/sql/pgwire/pgwirebase/encoding.go b/pkg/sql/pgwire/pgwirebase/encoding.go index 52dae906d0f9..8c99166bf6d6 100644 --- a/pkg/sql/pgwire/pgwirebase/encoding.go +++ b/pkg/sql/pgwire/pgwirebase/encoding.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "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/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/ipaddr" "github.com/cockroachdb/cockroach/pkg/util/timeofday" @@ -318,31 +319,33 @@ func DecodeDatum( ctx context.Context, evalCtx *eval.Context, typ *types.T, code FormatCode, b []byte, ) (tree.Datum, error) { id := typ.Oid() + // Use a direct string pointing to b where we are sure we aren't retaining this string. + bs := encoding.UnsafeConvertBytesToString(b) switch code { case FormatText: switch id { case oid.T_record: - d, _, err := tree.ParseDTupleFromString(evalCtx, string(b), typ) + d, _, err := tree.ParseDTupleFromString(evalCtx, bs, typ) if err != nil { return nil, err } return d, nil case oid.T_bool: - t, err := strconv.ParseBool(string(b)) + t, err := strconv.ParseBool(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.MakeDBool(tree.DBool(t)), nil case oid.T_bit, oid.T_varbit: - t, err := tree.ParseDBitArray(string(b)) + t, err := tree.ParseDBitArray(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return t, nil case oid.T_int2, oid.T_int4, oid.T_int8: - i, err := strconv.ParseInt(string(b), 10, 64) + i, err := strconv.ParseInt(bs, 10, 64) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.NewDInt(tree.DInt(i)), nil case oid.T_oid, @@ -356,106 +359,108 @@ func DecodeDatum( oid.T_regnamespace, oid.T_regprocedure, oid.T_regdictionary: - return eval.ParseDOid(ctx, evalCtx, string(b), typ) + return eval.ParseDOid(ctx, evalCtx, bs, typ) case oid.T_float4, oid.T_float8: - f, err := strconv.ParseFloat(string(b), 64) + f, err := strconv.ParseFloat(bs, 64) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return tree.NewDFloat(tree.DFloat(f)), nil case oidext.T_box2d: - d, err := tree.ParseDBox2D(string(b)) + d, err := tree.ParseDBox2D(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oidext.T_geography: - d, err := tree.ParseDGeography(string(b)) + d, err := tree.ParseDGeography(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oidext.T_geometry: - d, err := tree.ParseDGeometry(string(b)) + d, err := tree.ParseDGeometry(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_void: return tree.DVoidDatum, nil case oid.T_numeric: - d, err := tree.ParseDDecimal(string(b)) + d, err := tree.ParseDDecimal(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_bytea: res, err := lex.DecodeRawBytesToByteArrayAuto(b) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } - return tree.NewDBytes(tree.DBytes(res)), nil + // Note: we could use encoding.UnsafeConvertBytesToString here if + // we were guaranteed all callers never mutated b. + return tree.NewDBytes(tree.DBytes(string(res))), nil case oid.T_timestamp: - d, _, err := tree.ParseDTimestamp(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestamp(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_timestamptz: - d, _, err := tree.ParseDTimestampTZ(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimestampTZ(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_date: - d, _, err := tree.ParseDDate(evalCtx, string(b)) + d, _, err := tree.ParseDDate(evalCtx, bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_time: - d, _, err := tree.ParseDTime(nil, string(b), time.Microsecond) + d, _, err := tree.ParseDTime(nil, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_timetz: - d, _, err := tree.ParseDTimeTZ(evalCtx, string(b), time.Microsecond) + d, _, err := tree.ParseDTimeTZ(evalCtx, bs, time.Microsecond) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_interval: - d, err := tree.ParseDInterval(evalCtx.GetIntervalStyle(), string(b)) + d, err := tree.ParseDInterval(evalCtx.GetIntervalStyle(), bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_uuid: - d, err := tree.ParseDUuidFromString(string(b)) + d, err := tree.ParseDUuidFromString(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_inet: - d, err := tree.ParseDIPAddrFromINetString(string(b)) + d, err := tree.ParseDIPAddrFromINetString(bs) if err != nil { - return nil, tree.MakeParseError(string(b), typ, err) + return nil, tree.MakeParseError(bs, typ, err) } return d, nil case oid.T_jsonb, oid.T_json: if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(bs) case oid.T_tsquery: - ret, err := tsearch.ParseTSQuery(string(b)) + ret, err := tsearch.ParseTSQuery(bs) if err != nil { return nil, err } return &tree.DTSQuery{TSQuery: ret}, nil case oid.T_tsvector: - ret, err := tsearch.ParseTSVector(string(b)) + ret, err := tsearch.ParseTSVector(bs) if err != nil { return nil, err } @@ -467,7 +472,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.NewDString(string(b)), nil + return tree.NewDString(bs), nil } case FormatBinary: switch id { @@ -607,7 +612,7 @@ func DecodeDatum( decDigits = decDigits[:len(decDigits)-int(overScale)] } - decString := string(decDigits) + decString := encoding.UnsafeConvertBytesToString(decDigits) if _, ok := alloc.dd.Coeff.SetString(decString, 10); !ok { return nil, pgerror.Newf(pgcode.Syntax, "could not parse %q as type decimal", decString) } @@ -634,6 +639,7 @@ func DecodeDatum( return &alloc.dd, nil case oid.T_bytea: + // Note: there's an implicit string cast here reallocating b. return tree.NewDBytes(tree.DBytes(b)), nil case oid.T_timestamp: if len(b) < 8 { @@ -692,7 +698,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(bs) case oid.T_jsonb: if len(b) < 1 { return nil, NewProtocolViolationErrorf("no data to decode") @@ -705,7 +711,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - return tree.ParseDJSON(string(b)) + return tree.ParseDJSON(encoding.UnsafeConvertBytesToString(b)) case oid.T_varbit, oid.T_bit: if len(b) < 4 { return nil, NewProtocolViolationErrorf("insufficient data: %d", len(b)) @@ -795,7 +801,7 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } - e, err := tree.MakeDEnumFromLogicalRepresentation(typ, string(b)) + e, err := tree.MakeDEnumFromLogicalRepresentation(typ, bs) if err != nil { return nil, err } @@ -806,15 +812,21 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. return tree.NewDString(string(b)), nil case oid.T_bpchar: if err := validateStringBytes(b); err != nil { return nil, err } // Trim the trailing spaces + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. sv := strings.TrimRight(string(b), " ") return tree.NewDString(sv), nil case oid.T_char: + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. sv := string(b) // Always truncate to 1 byte, and handle the null byte specially. if len(b) >= 1 { @@ -829,6 +841,8 @@ func DecodeDatum( if err := validateStringBytes(b); err != nil { return nil, err } + // Note: we could use bs here if we were guaranteed all callers never + // mutated b. return tree.NewDName(string(b)), nil } From f3bd0bc47519c7b083200c5854802befb919615f Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 24 Apr 2023 08:49:53 -0400 Subject: [PATCH 10/12] sql: Avoid copy of file upload data All we do here is write the bytes to a stream so we don't need a copy. Release note: None Epic: None --- pkg/sql/copy_file_upload.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/copy_file_upload.go b/pkg/sql/copy_file_upload.go index 504fc83f1e2b..088fe49bca1d 100644 --- a/pkg/sql/copy_file_upload.go +++ b/pkg/sql/copy_file_upload.go @@ -194,7 +194,7 @@ func (f *fileUploadMachine) run(ctx context.Context) error { func (f *fileUploadMachine) writeFile(ctx context.Context, finalBatch bool) error { for i := 0; i < f.c.rows.Len(); i++ { r := f.c.rows.At(i) - b := []byte(*r[0].(*tree.DBytes)) + b := r[0].(*tree.DBytes).UnsafeBytes() n, err := f.w.Write(b) if err != nil { return err From 0cab4d4a00fcdffabf5498926d3f808c591c761e Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Mon, 24 Apr 2023 09:13:08 -0400 Subject: [PATCH 11/12] builtins: avoid multiple data copies in convert_from We copy the DBytes into a []byte and then copy the []byte into a string. Now we get a reference to the string and return that if appropriate or iterator over string bytes directly if not. This is all predicated on DBytes being immutable. Release note: None Epic: None --- pkg/sql/sem/builtins/builtins.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index eab59a5f97f5..2d0a8ddb02bb 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -430,20 +430,20 @@ var regularBuiltins = map[string]builtinDefinition{ Types: tree.ParamTypes{{Name: "str", Typ: types.Bytes}, {Name: "enc", Typ: types.String}}, ReturnType: tree.FixedReturnType(types.String), Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { - str := []byte(tree.MustBeDBytes(args[0])) + str := string(tree.MustBeDBytes(args[0])) enc := CleanEncodingName(string(tree.MustBeDString(args[1]))) switch enc { // All the following are aliases to each other in PostgreSQL. case "utf8", "unicode", "cp65001": - if !utf8.Valid(str) { + if !utf8.Valid(encoding.UnsafeConvertStringToBytes(str)) { return nil, newDecodeError("UTF8") } - return tree.NewDString(string(str)), nil + return tree.NewDString(str), nil // All the following are aliases to each other in PostgreSQL. case "latin1", "iso88591", "cp28591": var buf strings.Builder - for _, c := range str { + for _, c := range encoding.UnsafeConvertStringToBytes(str) { buf.WriteRune(rune(c)) } return tree.NewDString(buf.String()), nil From e67b8ed3e38b7bc32abc48b210af53764a0048a4 Mon Sep 17 00:00:00 2001 From: Tommy Reilly Date: Wed, 3 May 2023 12:45:21 -0400 Subject: [PATCH 12/12] json: avoid copying json.AsText where possible Epic: None Release note: None --- pkg/sql/colexec/colexecproj/BUILD.bazel | 1 + .../colexecproj/proj_non_const_ops.eg.go | 41 ++++++++++--------- .../colexecproj/proj_non_const_ops_tmpl.go | 1 + pkg/sql/colexec/colexecprojconst/BUILD.bazel | 1 + .../proj_const_left_ops.eg.go | 41 ++++++++++--------- .../colexecprojconst/proj_const_ops_tmpl.go | 1 + .../proj_const_right_ops.eg.go | 41 ++++++++++--------- .../execgen/cmd/execgen/overloads_bin.go | 6 +-- pkg/sql/sem/builtins/geo_builtins.go | 5 ++- pkg/sql/sem/eval/cast.go | 5 ++- 10 files changed, 76 insertions(+), 67 deletions(-) diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 1c9520beda64..6e78ce13a23b 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//pkg/sql/sqltelemetry", # keep "//pkg/sql/types", # keep "//pkg/util/duration", # keep + "//pkg/util/encoding", # keep "//pkg/util/json", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep "@com_github_cockroachdb_errors//:errors", # keep diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index 83bc025d8523..54249b5d2545 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) @@ -24176,7 +24177,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24210,7 +24211,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24242,7 +24243,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24272,7 +24273,7 @@ func (p projJSONFetchTextJSONBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24327,7 +24328,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24359,7 +24360,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24388,7 +24389,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24416,7 +24417,7 @@ func (p projJSONFetchTextJSONInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24471,7 +24472,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24503,7 +24504,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24532,7 +24533,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24560,7 +24561,7 @@ func (p projJSONFetchTextJSONInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24615,7 +24616,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24647,7 +24648,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24676,7 +24677,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24704,7 +24705,7 @@ func (p projJSONFetchTextJSONInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -24865,7 +24866,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24897,7 +24898,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24927,7 +24928,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -24955,7 +24956,7 @@ func (p projJSONFetchTextPathJSONDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index 730a4cee0724..0125456a41b4 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/errors" ) diff --git a/pkg/sql/colexec/colexecprojconst/BUILD.bazel b/pkg/sql/colexec/colexecprojconst/BUILD.bazel index d56bf6d9a6d8..8f671d992d04 100644 --- a/pkg/sql/colexec/colexecprojconst/BUILD.bazel +++ b/pkg/sql/colexec/colexecprojconst/BUILD.bazel @@ -34,6 +34,7 @@ go_library( "//pkg/sql/sqltelemetry", # keep "//pkg/sql/types", "//pkg/util/duration", # keep + "//pkg/util/encoding", # keep "//pkg/util/json", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go index f697fafc897a..f3e8a408d688 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -22395,7 +22396,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22426,7 +22427,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22457,7 +22458,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22485,7 +22486,7 @@ func (p projJSONFetchTextJSONConstBytesOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22540,7 +22541,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22568,7 +22569,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22596,7 +22597,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22621,7 +22622,7 @@ func (p projJSONFetchTextJSONConstInt16Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22676,7 +22677,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22704,7 +22705,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22732,7 +22733,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22757,7 +22758,7 @@ func (p projJSONFetchTextJSONConstInt32Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22812,7 +22813,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22840,7 +22841,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22868,7 +22869,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22893,7 +22894,7 @@ func (p projJSONFetchTextJSONConstInt64Op) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -23048,7 +23049,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23077,7 +23078,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23106,7 +23107,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23132,7 +23133,7 @@ func (p projJSONFetchTextPathJSONConstDatumOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index b9ad2c124e0f..58a7d8d5da73 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go index ce8dc945d82e..04012e174f75 100644 --- a/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/errors" ) @@ -22422,7 +22423,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22453,7 +22454,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22484,7 +22485,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22512,7 +22513,7 @@ func (p projJSONFetchTextJSONBytesConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22567,7 +22568,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22596,7 +22597,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22624,7 +22625,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22650,7 +22651,7 @@ func (p projJSONFetchTextJSONInt16ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22705,7 +22706,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22734,7 +22735,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22762,7 +22763,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22788,7 +22789,7 @@ func (p projJSONFetchTextJSONInt32ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22843,7 +22844,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22872,7 +22873,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22900,7 +22901,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -22926,7 +22927,7 @@ func (p projJSONFetchTextJSONInt64ConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } } @@ -23081,7 +23082,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23110,7 +23111,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23139,7 +23140,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } @@ -23165,7 +23166,7 @@ func (p projJSONFetchTextPathJSONDatumConstOp) Next() coldata.Batch { if _text == nil { _outNulls.SetNull(i) } else { - projCol.Set(i, []byte(*_text)) + projCol.Set(i, encoding.UnsafeConvertStringToBytes(*_text)) } } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go index f7a896d38cbe..2f896143e79c 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/overloads_bin.go @@ -684,7 +684,7 @@ if _j == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) }`, vecVariable, idxVariable)) case treebin.Minus: return fmt.Sprintf(` @@ -736,7 +736,7 @@ if _j == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) }`, vecVariable, idxVariable)) case treebin.Minus: return fmt.Sprintf(` @@ -787,7 +787,7 @@ if _path == nil { if _text == nil { _outNulls.SetNull(%[2]s) } else { - %[1]s.Set(%[2]s, []byte(*_text)) + %[1]s.Set(%[2]s, encoding.UnsafeConvertStringToBytes(*_text)) } `, vecVariable, idxVariable)) default: diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index 586a24416692..8005ca1e0160 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/storageparam" "github.com/cockroachdb/cockroach/pkg/sql/storageparam/indexstorageparam" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/intsets" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -760,7 +761,7 @@ var geoBuiltins = map[string]builtinDefinition{ if asString == nil { return tree.DNull, nil } - g, err := geo.ParseGeometryFromGeoJSON([]byte(*asString)) + g, err := geo.ParseGeometryFromGeoJSON(encoding.UnsafeConvertStringToBytes(*asString)) if err != nil { return nil, err } @@ -1063,7 +1064,7 @@ var geoBuiltins = map[string]builtinDefinition{ if asString == nil { return tree.DNull, nil } - g, err := geo.ParseGeographyFromGeoJSON([]byte(*asString)) + g, err := geo.ParseGeographyFromGeoJSON(encoding.UnsafeConvertStringToBytes(*asString)) if err != nil { return nil, err } diff --git a/pkg/sql/sem/eval/cast.go b/pkg/sql/sem/eval/cast.go index 2d64828d838a..fbefb0f43138 100644 --- a/pkg/sql/sem/eval/cast.go +++ b/pkg/sql/sem/eval/cast.go @@ -30,6 +30,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/timeofday" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate" @@ -615,7 +616,7 @@ func performCastWithoutPrecisionTruncation( if t == nil { return tree.DNull, nil } - g, err := geo.ParseGeographyFromGeoJSON([]byte(*t)) + g, err := geo.ParseGeographyFromGeoJSON(encoding.UnsafeConvertStringToBytes(*t)) if err != nil { return nil, err } @@ -663,7 +664,7 @@ func performCastWithoutPrecisionTruncation( if t == nil { return tree.DNull, nil } - g, err := geo.ParseGeometryFromGeoJSON([]byte(*t)) + g, err := geo.ParseGeometryFromGeoJSON(encoding.UnsafeConvertStringToBytes(*t)) if err != nil { return nil, err }