From 40da6216f5c279817adacfebb5ce5c902b27e982 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Wed, 18 Dec 2024 15:31:00 -0800 Subject: [PATCH] wip --- pkg/sql/catalog/catenumpb/encoded_datum.proto | 2 + pkg/sql/row/fetcher.go | 21 +++----- pkg/sql/rowenc/encoded_datum.go | 50 +++++++++++++++++++ pkg/sql/rowenc/encoded_datum_test.go | 25 ++++++++-- 4 files changed, 81 insertions(+), 17 deletions(-) diff --git a/pkg/sql/catalog/catenumpb/encoded_datum.proto b/pkg/sql/catalog/catenumpb/encoded_datum.proto index 4db856d3cf8e..9ca508506ce7 100644 --- a/pkg/sql/catalog/catenumpb/encoded_datum.proto +++ b/pkg/sql/catalog/catenumpb/encoded_datum.proto @@ -20,4 +20,6 @@ enum DatumEncoding { DESCENDING_KEY = 1; // Indicates that the datum is encoded using the encoding used for values. VALUE = 2; + // Indicates that the datum is encoded using the legacy encoding for values. + VALUE_LEGACY = 3; } diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 58f14768e635..769833112dbf 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" - "github.com/cockroachdb/cockroach/pkg/sql/rowenc/valueside" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/scrub" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -1076,21 +1075,17 @@ func (rf *Fetcher) processValueSingle( return prettyKey, "", nil } typ := table.spec.FetchedColumns[idx].Type - // TODO(arjun): The value is a directly marshaled single value, so we - // unmarshal it eagerly here. This can potentially be optimized out, - // although that would require changing UnmarshalColumnValue to operate - // on bytes, and for Encode/DecodeTableValue to operate on marshaled - // single values. - value, err := valueside.UnmarshalLegacy(rf.args.Alloc, typ, kv.Value) - if err != nil { - return "", "", err - } + encValue := rowenc.EncDatumValueLegacyFromValue(kv.Value) if rf.args.TraceKV { - prettyValue = value.String() + err := encValue.EnsureDecoded(typ, rf.args.Alloc) + if err != nil { + return "", "", err + } + prettyValue = encValue.Datum.String() } - table.row[idx] = rowenc.DatumToEncDatum(typ, value) + table.row[idx] = encValue if DebugRowFetch { - log.Infof(ctx, "Scan %s -> %v", kv.Key, value) + log.Infof(ctx, "Scan %d -> %v", kv.Key, encValue) } return prettyKey, prettyValue, nil } diff --git a/pkg/sql/rowenc/encoded_datum.go b/pkg/sql/rowenc/encoded_datum.go index 2baa3f49499c..11da8f7b4eb5 100644 --- a/pkg/sql/rowenc/encoded_datum.go +++ b/pkg/sql/rowenc/encoded_datum.go @@ -11,6 +11,7 @@ import ( "fmt" "unsafe" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/keyside" @@ -158,6 +159,11 @@ func EncDatumFromBuffer(enc catenumpb.DatumEncoding, buf []byte) (EncDatum, []by } ed := EncDatumFromEncoded(enc, buf[typeOffset:encLen]) return ed, buf[encLen:], nil + case catenumpb.DatumEncoding_VALUE_LEGACY: + // The legacy value encoding is only used for individual datums occupying + // entire roachpb.Values, and therefore we don't expect to call + // EncDatumFromBuffer with the legacy encoding. + panic(errors.AssertionFailedf("EncDatumFromBuffer unimplemented for VALUE_LEGACY")) default: panic(errors.AssertionFailedf("unknown encoding %s", enc)) } @@ -179,6 +185,17 @@ func EncDatumValueFromBufferWithOffsetsAndType( return ed, buf[encLen:], nil } +// EncDatumValueLegacyFromValue +func EncDatumValueLegacyFromValue(value roachpb.Value) EncDatum { + if value.RawBytes == nil { + return EncDatum{ + encoding: catenumpb.DatumEncoding_VALUE_LEGACY, + Datum: tree.DNull, + } + } + return EncDatumFromEncoded(catenumpb.DatumEncoding_VALUE_LEGACY, value.TagAndDataBytes()) +} + // DatumToEncDatum initializes an EncDatum with the given Datum. func DatumToEncDatum(ctyp *types.T, d tree.Datum) EncDatum { ed, err := DatumToEncDatumEx(ctyp, d) @@ -242,6 +259,9 @@ func (ed *EncDatum) IsNull() bool { } return typ == encoding.Null + case catenumpb.DatumEncoding_VALUE_LEGACY: + return ed.encoded == nil + default: panic(errors.AssertionFailedf("unknown encoding %s", ed.encoding)) } @@ -264,6 +284,16 @@ func (ed *EncDatum) EnsureDecoded(typ *types.T, a *tree.DatumAlloc) error { ed.Datum, rem, err = keyside.Decode(a, typ, ed.encoded, encoding.Descending) case catenumpb.DatumEncoding_VALUE: ed.Datum, rem, err = valueside.Decode(a, typ, ed.encoded) + case catenumpb.DatumEncoding_VALUE_LEGACY: + var v roachpb.Value + v.SetTagAndData(ed.encoded) + // NULL is not stored in legacy-encoded values. We use an UNKNOWN tag to + // satisfy the EncDatum invariants that encoded != nil, and must check for + // that here. + if v.GetTag() == roachpb.ValueType_UNKNOWN { + v.RawBytes = nil + } + ed.Datum, err = valueside.UnmarshalLegacy(a, typ, v) default: return errors.AssertionFailedf("unknown encoding %d", redact.Safe(ed.encoding)) } @@ -308,6 +338,21 @@ func (ed *EncDatum) Encode( return keyside.Encode(appendTo, ed.Datum, encoding.Descending) case catenumpb.DatumEncoding_VALUE: return valueside.Encode(appendTo, valueside.NoColumnID, ed.Datum) + case catenumpb.DatumEncoding_VALUE_LEGACY: + // The legacy value encoding is only used for individual datums occupying + // entire roachpb.Values. Because of this, we don't expect to call Encode + // with the legacy encoding, bue we implement it anyway for testing + // purposes. + v, err := valueside.MarshalLegacy(typ, ed.Datum) + if err != nil { + return nil, err + } + // NULL is not stored in legacy-encoded values. We use an UNKNOWN tag here + // to satisfy the EncDatum invariants that encoded != nil. + if v.RawBytes == nil { + v.SetTag(roachpb.ValueType_UNKNOWN) + } + return v.TagAndDataBytes(), nil default: panic(errors.AssertionFailedf("unknown encoding requested %s", enc)) } @@ -466,6 +511,11 @@ func (ed *EncDatum) GetInt() (int64, error) { _, val, err := encoding.DecodeUntaggedIntValue(ed.encoded[dataOffset:]) return val, err + case catenumpb.DatumEncoding_VALUE_LEGACY: + var v roachpb.Value + v.SetTagAndData(ed.encoded) + return v.GetInt() + default: return 0, errors.Errorf("unknown encoding %s", ed.encoding) } diff --git a/pkg/sql/rowenc/encoded_datum_test.go b/pkg/sql/rowenc/encoded_datum_test.go index b614cf2ebbf3..c5f4e3b982c9 100644 --- a/pkg/sql/rowenc/encoded_datum_test.go +++ b/pkg/sql/rowenc/encoded_datum_test.go @@ -12,6 +12,7 @@ import ( "unsafe" "github.com/cockroachdb/apd/v3" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -116,7 +117,9 @@ func TestEncDatum(t *testing.T) { } func columnTypeCompatibleWithEncoding(typ *types.T, enc catenumpb.DatumEncoding) bool { - return enc == catenumpb.DatumEncoding_VALUE || colinfo.ColumnTypeIsIndexable(typ) + return enc == catenumpb.DatumEncoding_VALUE || + enc == catenumpb.DatumEncoding_VALUE_LEGACY || + colinfo.ColumnTypeIsIndexable(typ) } func TestEncDatumNull(t *testing.T) { @@ -245,6 +248,7 @@ func TestEncDatumCompare(t *testing.T) { asc := catenumpb.DatumEncoding_ASCENDING_KEY desc := catenumpb.DatumEncoding_DESCENDING_KEY noncmp := catenumpb.DatumEncoding_VALUE + legacy := catenumpb.DatumEncoding_VALUE_LEGACY checkEncDatumCmp(t, a, typ, &v1, &v2, asc, asc, -1, false) checkEncDatumCmp(t, a, typ, &v2, &v1, asc, asc, +1, false) @@ -263,6 +267,10 @@ func TestEncDatumCompare(t *testing.T) { checkEncDatumCmp(t, a, typ, &v2, &v1, desc, noncmp, +1, true) checkEncDatumCmp(t, a, typ, &v1, &v1, asc, desc, 0, true) checkEncDatumCmp(t, a, typ, &v2, &v2, desc, asc, 0, true) + checkEncDatumCmp(t, a, typ, &v1, &v2, noncmp, legacy, -1, true) + checkEncDatumCmp(t, a, typ, &v2, &v1, desc, legacy, +1, true) + checkEncDatumCmp(t, a, typ, &v1, &v1, asc, legacy, 0, true) + checkEncDatumCmp(t, a, typ, &v2, &v2, legacy, legacy, 0, true) } } } @@ -714,14 +722,18 @@ func TestEncDatumFingerprintMemory(t *testing.T) { defer leaktest.AfterTest(t)() const ( - asc = catenumpb.DatumEncoding_ASCENDING_KEY - desc = catenumpb.DatumEncoding_DESCENDING_KEY - value = catenumpb.DatumEncoding_VALUE + asc = catenumpb.DatumEncoding_ASCENDING_KEY + desc = catenumpb.DatumEncoding_DESCENDING_KEY + value = catenumpb.DatumEncoding_VALUE + legacy = catenumpb.DatumEncoding_VALUE_LEGACY i = 123 s = "abcde" ) + var v roachpb.Value + v.SetString(s) + testCases := []struct { encDatum rowenc.EncDatum typ *types.T @@ -744,6 +756,11 @@ func TestEncDatumFingerprintMemory(t *testing.T) { typ: types.String, newMemUsage: int64(tree.NewDString(s).Size()), }, + { + encDatum: rowenc.EncDatumFromEncoded(legacy, v.TagAndDataBytes()), + typ: types.String, + newMemUsage: int64(tree.NewDString(s).Size()), + }, } ctx := context.Background()