From 45bd97c4b3cec845a969f31b25ab1d0d45373c2a Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 19 Dec 2024 19:37:52 -0800 Subject: [PATCH] sql: replace InitPut with CPut CPut with empty `expValue` parameter is the same as InitPut (with `failOnTombstones=false`) when we don't expect the key to exist, so we replace the latter with the former throughout the code base. If the key might already exist, then we need to use CPut with the possible value and `AllowIfDoesNotExist` set to `true` (which is the case for the backfill code where the value might exist when performing the primary key change). This paves the way to deprecating the InitPut request altogether. Release note: None --- .../partitioning_implicit_read_committed | 6 ++ pkg/kv/batch.go | 72 ++++++--------- pkg/kv/db_test.go | 3 +- pkg/sql/backfill/backfill.go | 8 +- pkg/sql/catalog/ingesting/write_descs.go | 2 +- pkg/sql/catalog/tabledesc/index_test.go | 4 +- pkg/sql/colenc/bench_test.go | 4 +- pkg/sql/colenc/encode.go | 6 +- pkg/sql/colenc/encode_test.go | 41 +++------ pkg/sql/colenc/inverted.go | 2 +- pkg/sql/copy/testdata/copy_from | 18 ++-- pkg/sql/copy_from.go | 2 +- .../opt/exec/execbuilder/testdata/geospatial | 24 ++--- .../exec/execbuilder/testdata/inverted_index | 50 +++++------ .../testdata/inverted_index_multi_column | 36 ++++---- pkg/sql/opt/exec/execbuilder/testdata/json | 10 +-- .../exec/execbuilder/testdata/partial_index | 90 +++++++++---------- .../testdata/secondary_index_column_families | 64 ++++++------- .../opt/exec/execbuilder/testdata/show_trace | 6 +- pkg/sql/opt/exec/execbuilder/testdata/upsert | 4 +- pkg/sql/row/inserter.go | 4 +- pkg/sql/row/putter.go | 80 +++++------------ pkg/sql/row/row_converter.go | 24 +++-- pkg/sql/tests/sysbench_test.go | 2 +- pkg/upgrade/upgrades/descriptor_utils.go | 2 +- 25 files changed, 251 insertions(+), 313 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed index 1079a56e4d1f..52bc466321c5 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit_read_committed @@ -404,6 +404,7 @@ CPut /Table/110/1/" "/1/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xa0"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/1/0 -> nil (tombstone) +CPut /Table/110/2/"@"/4/0 -> /BYTES/0x89 CPut /Table/110/2/" "/4/0 -> nil (tombstone) CPut /Table/110/2/"\x80"/4/0 -> nil (tombstone) CPut /Table/110/2/"\xa0"/4/0 -> nil (tombstone) @@ -413,6 +414,7 @@ CPut /Table/110/1/"@"/1/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xa0"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/1/0 -> nil (tombstone) +CPut /Table/110/2/" "/6/0 -> /BYTES/0x89 CPut /Table/110/2/"@"/6/0 -> nil (tombstone) CPut /Table/110/2/"\x80"/6/0 -> nil (tombstone) CPut /Table/110/2/"\xa0"/6/0 -> nil (tombstone) @@ -422,6 +424,7 @@ CPut /Table/110/1/" "/2/0 -> nil (tombstone) CPut /Table/110/1/"@"/2/0 -> nil (tombstone) CPut /Table/110/1/"\xa0"/2/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/2/0 -> nil (tombstone) +CPut /Table/110/2/"\x80"/4/0 -> /BYTES/0x8a CPut /Table/110/2/" "/4/0 -> nil (tombstone) CPut /Table/110/2/"@"/4/0 -> nil (tombstone) CPut /Table/110/2/"\xa0"/4/0 -> nil (tombstone) @@ -431,6 +434,7 @@ CPut /Table/110/1/" "/2/0 -> nil (tombstone) CPut /Table/110/1/"@"/2/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/2/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/2/0 -> nil (tombstone) +CPut /Table/110/2/"\xa0"/6/0 -> /BYTES/0x8a CPut /Table/110/2/" "/6/0 -> nil (tombstone) CPut /Table/110/2/"@"/6/0 -> nil (tombstone) CPut /Table/110/2/"\x80"/6/0 -> nil (tombstone) @@ -440,6 +444,7 @@ CPut /Table/110/1/" "/1/0 -> nil (tombstone) CPut /Table/110/1/"@"/1/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xc0"/1/0 -> nil (tombstone) +CPut /Table/110/2/"\xa0"/6/0 -> /BYTES/0x89 CPut /Table/110/2/"\xa0"/4/0 -> /BYTES/0x8a (expecting does not exist) CPut /Table/110/2/" "/4/0 -> nil (tombstone) CPut /Table/110/2/"@"/4/0 -> nil (tombstone) @@ -450,6 +455,7 @@ CPut /Table/110/1/" "/1/0 -> nil (tombstone) CPut /Table/110/1/"@"/1/0 -> nil (tombstone) CPut /Table/110/1/"\x80"/1/0 -> nil (tombstone) CPut /Table/110/1/"\xa0"/1/0 -> nil (tombstone) +CPut /Table/110/2/"\xc0"/4/0 -> /BYTES/0x89 CPut /Table/110/2/" "/4/0 -> nil (tombstone) CPut /Table/110/2/"@"/4/0 -> nil (tombstone) CPut /Table/110/2/"\x80"/4/0 -> nil (tombstone) diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index e3ab7fb9708f..56eefac61726 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -625,6 +625,32 @@ func (b *Batch) cputInternal( b.initResult(1, 1, notRaw, nil) } +// CPutBytesEmpty allows multiple []byte value type CPut requests to be added to +// the batch using BulkSource interface. The values for these keys are +// expected to be empty. +func (b *Batch) CPutBytesEmpty(bs BulkSource[[]byte]) { + numKeys := bs.Len() + reqs := make([]struct { + req kvpb.ConditionalPutRequest + union kvpb.RequestUnion_ConditionalPut + }, numKeys) + i := 0 + bsi := bs.Iter() + b.bulkRequest(numKeys, func() (kvpb.RequestUnion, int) { + pr := &reqs[i].req + union := &reqs[i].union + union.ConditionalPut = pr + pr.AllowIfDoesNotExist = false + pr.ExpBytes = nil + i++ + k, v := bsi.Next() + pr.Key = k + pr.Value.SetBytes(v) + pr.Value.InitChecksum(k) + return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes) + }) +} + // CPutTuplesEmpty allows multiple CPut tuple requests to be added to the batch // as tuples using the BulkSource interface. The values for these keys are // expected to be empty. @@ -701,52 +727,6 @@ func (b *Batch) InitPut(key, value interface{}, failOnTombstones bool) { b.initResult(1, 1, notRaw, nil) } -// InitPutBytes allows multiple []byte value type InitPut requests to be added to -// the batch using BulkSource interface. -func (b *Batch) InitPutBytes(bs BulkSource[[]byte]) { - numKeys := bs.Len() - reqs := make([]struct { - req kvpb.InitPutRequest - union kvpb.RequestUnion_InitPut - }, numKeys) - i := 0 - bsi := bs.Iter() - b.bulkRequest(numKeys, func() (kvpb.RequestUnion, int) { - pr := &reqs[i].req - union := &reqs[i].union - union.InitPut = pr - i++ - k, v := bsi.Next() - pr.Key = k - pr.Value.SetBytes(v) - pr.Value.InitChecksum(k) - return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes) - }) -} - -// InitPutTuples allows multiple tuple value type InitPut to be added to the -// batch using BulkSource interface. -func (b *Batch) InitPutTuples(bs BulkSource[[]byte]) { - numKeys := bs.Len() - reqs := make([]struct { - req kvpb.InitPutRequest - union kvpb.RequestUnion_InitPut - }, numKeys) - i := 0 - bsi := bs.Iter() - b.bulkRequest(numKeys, func() (kvpb.RequestUnion, int) { - pr := &reqs[i].req - union := &reqs[i].union - union.InitPut = pr - i++ - k, v := bsi.Next() - pr.Key = k - pr.Value.SetTuple(v) - pr.Value.InitChecksum(k) - return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes) - }) -} - // Inc increments the integer value at key. If the key does not exist it will // be created with an initial value of 0 which will then be incremented. If the // key exists but was set using Put or CPut an error will be returned. diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index 0e80e3f41266..cc5dee62ff15 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -1138,8 +1138,7 @@ func TestBulkBatchAPI(t *testing.T) { testF(func(b *kv.Batch) { b.PutBytes(&byteSliceBulkSource[[]byte]{kys, vals}) }) testF(func(b *kv.Batch) { b.PutTuples(&byteSliceBulkSource[[]byte]{kys, vals}) }) - testF(func(b *kv.Batch) { b.InitPutBytes(&byteSliceBulkSource[[]byte]{kys, vals}) }) - testF(func(b *kv.Batch) { b.InitPutTuples(&byteSliceBulkSource[[]byte]{kys, vals}) }) + testF(func(b *kv.Batch) { b.CPutBytesEmpty(&byteSliceBulkSource[[]byte]{kys, vals}) }) testF(func(b *kv.Batch) { b.CPutTuplesEmpty(&byteSliceBulkSource[[]byte]{kys, vals}) }) values := make([]roachpb.Value, len(kys)) diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 8aada612e700..ba47ddf8d58f 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -1046,9 +1046,13 @@ func (ib *IndexBackfiller) RunIndexBackfillChunk( for _, entry := range entries { if traceKV { - log.VEventf(ctx, 2, "InitPut %s -> %s", entry.Key, entry.Value.PrettyPrint()) + log.VEventf(ctx, 2, "CPut %s -> %s", entry.Key, entry.Value.PrettyPrint()) } - batch.InitPut(entry.Key, &entry.Value, false /* failOnTombstones */) + // Note that we generally don't expect the previous value to exist, so + // CPut with nil expValue would be sufficient. This is not the case when + // performing the primary key change where the previous value might + // exist. + batch.CPutAllowingIfNotExists(entry.Key, &entry.Value, entry.Value.TagAndDataBytes()) } writeBatch := txn.Run if alsoCommit { diff --git a/pkg/sql/catalog/ingesting/write_descs.go b/pkg/sql/catalog/ingesting/write_descs.go index b3403af1fd76..b0fbec595ba8 100644 --- a/pkg/sql/catalog/ingesting/write_descs.go +++ b/pkg/sql/catalog/ingesting/write_descs.go @@ -214,7 +214,7 @@ func WriteDescriptors( } for _, kv := range extra { - b.InitPut(kv.Key, &kv.Value, false) + b.CPut(kv.Key, &kv.Value, nil /* expValue */) } if err := txn.Run(ctx, b); err != nil { if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { diff --git a/pkg/sql/catalog/tabledesc/index_test.go b/pkg/sql/catalog/tabledesc/index_test.go index 2ea14691cdd5..3ec271ec3228 100644 --- a/pkg/sql/catalog/tabledesc/index_test.go +++ b/pkg/sql/catalog/tabledesc/index_test.go @@ -381,7 +381,7 @@ func TestIndexStrictColumnIDs(t *testing.T) { require.NoError(t, err) // Retrieve KV trace and check for redundant values. - rows, err := conn.Query(`SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'InitPut%'`) + rows, err := conn.Query(`SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'CPut%/Table/%/2% ->%'`) require.NoError(t, err) defer rows.Close() require.True(t, rows.Next()) @@ -392,7 +392,7 @@ func TestIndexStrictColumnIDs(t *testing.T) { if srv.TenantController().StartedDefaultTestTenant() { tenantPrefix = codec.TenantPrefix().String() } - expected := fmt.Sprintf(`InitPut %s/Table/%d/2/0/0/0/0/0/0 -> /BYTES/0x2300030003000300`, tenantPrefix, mut.GetID()) + expected := fmt.Sprintf(`CPut %s/Table/%d/2/0/0/0/0/0/0 -> /BYTES/0x2300030003000300`, tenantPrefix, mut.GetID()) require.Equal(t, expected, msg) // Test that with the strict guarantees, this table descriptor would have been diff --git a/pkg/sql/colenc/bench_test.go b/pkg/sql/colenc/bench_test.go index ebf773dadf45..f78d4a5d664a 100644 --- a/pkg/sql/colenc/bench_test.go +++ b/pkg/sql/colenc/bench_test.go @@ -142,11 +142,9 @@ func (n *noopPutter) CPutWithOriginTimestamp( ) { } func (n *noopPutter) Put(key, value interface{}) {} -func (n *noopPutter) InitPut(key, value interface{}, failOnTombstones bool) {} func (n *noopPutter) Del(key ...interface{}) {} +func (n *noopPutter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) {} func (n *noopPutter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) {} func (n *noopPutter) CPutTuplesEmpty(kys []roachpb.Key, values [][]byte) {} func (n *noopPutter) PutBytes(kys []roachpb.Key, values [][]byte) {} -func (n *noopPutter) InitPutBytes(kys []roachpb.Key, values [][]byte) {} func (n *noopPutter) PutTuples(kys []roachpb.Key, values [][]byte) {} -func (n *noopPutter) InitPutTuples(kys []roachpb.Key, values [][]byte) {} diff --git a/pkg/sql/colenc/encode.go b/pkg/sql/colenc/encode.go index 4fb14c579814..318d12fe27f4 100644 --- a/pkg/sql/colenc/encode.go +++ b/pkg/sql/colenc/encode.go @@ -494,7 +494,7 @@ func (b *BatchEncoder) encodeSecondaryIndexNoFamilies(ind catalog.Index, kys []r if err := b.writeColumnValues(kys, values, ind, cols); err != nil { return err } - b.p.InitPutBytes(kys, values) + b.p.CPutBytesEmpty(kys, values) return nil } @@ -556,9 +556,9 @@ func (b *BatchEncoder) encodeSecondaryIndexWithFamilies( // include encoded primary key columns. For other families, // use the tuple encoding for the value. if familyID == 0 { - b.p.InitPutBytes(kys, values) + b.p.CPutBytesEmpty(kys, values) } else { - b.p.InitPutTuples(kys, values) + b.p.CPutTuplesEmpty(kys, values) } if err := b.checkMemory(); err != nil { return err diff --git a/pkg/sql/colenc/encode_test.go b/pkg/sql/colenc/encode_test.go index c6dee5ea04f4..e9d1cb083abc 100644 --- a/pkg/sql/colenc/encode_test.go +++ b/pkg/sql/colenc/encode_test.go @@ -720,17 +720,22 @@ func (c *capturePutter) Put(key, value interface{}) { c.kvs.values = append(c.kvs.values, copyBytes(v.RawBytes)) } -func (c *capturePutter) InitPut(key, value interface{}, failOnTombstones bool) { - k := key.(*roachpb.Key) - c.kvs.keys = append(c.kvs.keys, *k) - v := value.(*roachpb.Value) - c.kvs.values = append(c.kvs.values, copyBytes(v.RawBytes)) -} - func (c *capturePutter) Del(key ...interface{}) { colexecerror.InternalError(errors.New("unimplemented")) } +func (c *capturePutter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) { + for i, k := range kys { + if len(k) == 0 { + continue + } + c.kvs.keys = append(c.kvs.keys, k) + var kvValue roachpb.Value + kvValue.SetBytes(values[i]) + c.kvs.values = append(c.kvs.values, kvValue.RawBytes) + } +} + func (c *capturePutter) CPutTuplesEmpty(kys []roachpb.Key, values [][]byte) { for i, k := range kys { if len(k) == 0 { @@ -757,33 +762,11 @@ func (c *capturePutter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Valu func (c *capturePutter) PutBytes(kys []roachpb.Key, values [][]byte) { colexecerror.InternalError(errors.New("unimplemented")) } -func (c *capturePutter) InitPutBytes(kys []roachpb.Key, values [][]byte) { - for i, k := range kys { - if len(k) == 0 { - continue - } - c.kvs.keys = append(c.kvs.keys, k) - var kvValue roachpb.Value - kvValue.SetBytes(values[i]) - c.kvs.values = append(c.kvs.values, kvValue.RawBytes) - } -} // we don't call this func (c *capturePutter) PutTuples(kys []roachpb.Key, values [][]byte) { colexecerror.InternalError(errors.New("unimplemented")) } -func (c *capturePutter) InitPutTuples(kys []roachpb.Key, values [][]byte) { - for i, k := range kys { - if len(k) == 0 { - continue - } - c.kvs.keys = append(c.kvs.keys, k) - var kvValue roachpb.Value - kvValue.SetTuple(values[i]) - c.kvs.values = append(c.kvs.values, kvValue.RawBytes) - } -} type kvs struct { keys []roachpb.Key diff --git a/pkg/sql/colenc/inverted.go b/pkg/sql/colenc/inverted.go index 7e0565883008..b2860fad6645 100644 --- a/pkg/sql/colenc/inverted.go +++ b/pkg/sql/colenc/inverted.go @@ -94,7 +94,7 @@ func (b *BatchEncoder) encodeInvertedSecondaryIndexNoFamiliesOneRow( } var kvValue roachpb.Value kvValue.SetBytes(value) - b.p.InitPut(&key, &kvValue, false) + b.p.CPut(&key, &kvValue, nil /* expValue */) return b.checkMemory() } diff --git a/pkg/sql/copy/testdata/copy_from b/pkg/sql/copy/testdata/copy_from index b381413ca584..0597fa261786 100644 --- a/pkg/sql/copy/testdata/copy_from +++ b/pkg/sql/copy/testdata/copy_from @@ -557,7 +557,7 @@ COPY tpartial FROM STDIN CPut /Table/<>/1/-1/0 -> /TUPLE/ CPut /Table/<>/1/0/0 -> /TUPLE/ CPut /Table/<>/1/1/0 -> /TUPLE/ -InitPut /Table/<>/2/1/0 -> /BYTES/ +CPut /Table/<>/2/1/0 -> /BYTES/ exec-ddl CREATE TABLE tpartial2 (i INT PRIMARY KEY, b INT, index(b) WHERE i > 0, FAMILY (i), FAMILY (b)) @@ -585,7 +585,7 @@ COPY tpartial2 FROM STDIN ---- CPut /Table/<>/1/1/0 -> /TUPLE/ CPut /Table/<>/1/1/1/1 -> /INT/2 -InitPut /Table/<>/2/2/1/0 -> /BYTES/ +CPut /Table/<>/2/2/1/0 -> /BYTES/ exec-ddl CREATE TYPE testenum AS ENUM('cat','dog','bear'); @@ -625,7 +625,7 @@ COPY tenum2 FROM STDIN 1 cat ---- CPut /Table/<>/1/1/0 -> /TUPLE/2:2:Bytes/@ -InitPut /Table/<>/2/"@"/1/0 -> /BYTES/ +CPut /Table/<>/2/"@"/1/0 -> /BYTES/ exec-ddl CREATE TABLE tenum3 (i INT PRIMARY KEY, c1 testenum, UNIQUE INDEX(c1)) @@ -642,7 +642,7 @@ COPY tenum3 FROM STDIN 1 dog ---- CPut /Table/<>/1/1/0 -> /TUPLE/2:2:Bytes/0x80 -InitPut /Table/<>/2/"\x80"/0 -> /BYTES/0x89 +CPut /Table/<>/2/"\x80"/0 -> /BYTES/0x89 exec-ddl CREATE TYPE comp AS (a INT, b INT); @@ -702,9 +702,9 @@ CPut /Table/<>/1/2/2/1 -> /INT/3 CPut /Table/<>/1/3/0 -> /TUPLE/2:2:Int/5 CPut /Table/<>/1/3/1/1 -> /INT/2 CPut /Table/<>/1/3/2/1 -> /INT/1 -InitPut /Table/<>/2/1/2/4/1/0 -> /BYTES/ -InitPut /Table/<>/2/2/1/3/2/0 -> /BYTES/ -InitPut /Table/<>/2/2/5/1/3/0 -> /BYTES/ +CPut /Table/<>/2/1/2/4/1/0 -> /BYTES/ +CPut /Table/<>/2/2/1/3/2/0 -> /BYTES/ +CPut /Table/<>/2/2/5/1/3/0 -> /BYTES/ query SELECT * FROM tfam2 @@ -757,5 +757,5 @@ CPut /Table/<>/1/1/0 -> /TUPLE/2:2:Bytes/running CPut /Table/<>/1/1/1/1 -> /INT/3 CPut /Table/<>/1/2/0 -> /TUPLE/2:2:Bytes/succeed CPut /Table/<>/1/2/1/1 -> /INT/1 -InitPut /Table/<>/2/"running"/1/0 -> /BYTES/ -InitPut /Table/<>/2/"running"/1/1/1 -> /TUPLE/3:3:Int/3 +CPut /Table/<>/2/"running"/1/0 -> /BYTES/ +CPut /Table/<>/2/"running"/1/1/1 -> /TUPLE/3:3:Int/3 diff --git a/pkg/sql/copy_from.go b/pkg/sql/copy_from.go index 6aa8470b8f42..5e69b52d5f6b 100644 --- a/pkg/sql/copy_from.go +++ b/pkg/sql/copy_from.go @@ -360,7 +360,7 @@ func newCopyMachine( // we still have all the encoder allocations to make. // // We also make the fraction depend on the number of indexes in the table - // since each secondary index will require a separate InitPut command for + // since each secondary index will require a separate CPut command for // each input row. We want to pick the fraction to be in [0.1, 0.33] range // so that 0.33 is used with no secondary indexes and 0.1 is used with 16 or // more secondary indexes. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/geospatial b/pkg/sql/opt/exec/execbuilder/testdata/geospatial index 03fbf65c8703..7b75e9eb4bc3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/geospatial +++ b/pkg/sql/opt/exec/execbuilder/testdata/geospatial @@ -37,13 +37,13 @@ INSERT INTO c VALUES ---- Scan /Table/20/1/10{7-8} CPut /Table/107/1/1/0 -> /TUPLE/ -InitPut /Table/107/2/"B\xfd\x10\x01D\x15@\x80K\xd5\x01?\x91\xdfF\xa2R\x9d9?\x91\xdfF\xa2R\x9d9\x89\x88" -> /BYTES/ -InitPut /Table/107/3/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01@\x00\x00\x00\x00\x00\x00\x00@\x00\x00\x00\x00\x00\x00\x00\x89\x88" -> /BYTES/ +CPut /Table/107/2/"B\xfd\x10\x01D\x15@\x80K\xd5\x01?\x91\xdfF\xa2R\x9d9?\x91\xdfF\xa2R\x9d9\x89\x88" -> /BYTES/ +CPut /Table/107/3/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01@\x00\x00\x00\x00\x00\x00\x00@\x00\x00\x00\x00\x00\x00\x00\x89\x88" -> /BYTES/ CPut /Table/107/1/2/0 -> /TUPLE/ -InitPut /Table/107/2/"B\xfd\x10\x01P\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ -InitPut /Table/107/2/"B\xfd\x10\x03\xff\xff\xfc\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ -InitPut /Table/107/2/"B\xfd\x10\x05\x00\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ -InitPut /Table/107/3/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01?\xf0\x00\x00\x00\x00\x00\x00?\xf0\x00\x00\x00\x00\x00\x00\x8a\x88" -> /BYTES/ +CPut /Table/107/2/"B\xfd\x10\x01P\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ +CPut /Table/107/2/"B\xfd\x10\x03\xff\xff\xfc\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ +CPut /Table/107/2/"B\xfd\x10\x05\x00\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8a\x88" -> /BYTES/ +CPut /Table/107/3/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01?\xf0\x00\x00\x00\x00\x00\x00?\xf0\x00\x00\x00\x00\x00\x00\x8a\x88" -> /BYTES/ statement ok CREATE INVERTED INDEX geog_idx ON b(geog) @@ -57,13 +57,13 @@ INSERT INTO b VALUES (4, 'LINESTRING(1.0 1.0, 2.0 2.0)', 'POINT(1.0 1.0)') ---- CPut /Table/106/1/3/0 -> /TUPLE/ -InitPut /Table/106/2/"B\xfd\x10\x01D\x15@\x80K\xd5\x01?\x91\xdfF\xa2R\x9d9?\x91\xdfF\xa2R\x9d9\x8b\x88" -> /BYTES/ -InitPut /Table/106/4/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01@\x00\x00\x00\x00\x00\x00\x00@\x00\x00\x00\x00\x00\x00\x00\x8b\x88" -> /BYTES/ +CPut /Table/106/2/"B\xfd\x10\x01D\x15@\x80K\xd5\x01?\x91\xdfF\xa2R\x9d9?\x91\xdfF\xa2R\x9d9\x8b\x88" -> /BYTES/ +CPut /Table/106/4/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01@\x00\x00\x00\x00\x00\x00\x00@\x00\x00\x00\x00\x00\x00\x00\x8b\x88" -> /BYTES/ CPut /Table/106/1/4/0 -> /TUPLE/ -InitPut /Table/106/2/"B\xfd\x10\x01P\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ -InitPut /Table/106/2/"B\xfd\x10\x03\xff\xff\xfc\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ -InitPut /Table/106/2/"B\xfd\x10\x05\x00\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ -InitPut /Table/106/4/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01?\xf0\x00\x00\x00\x00\x00\x00?\xf0\x00\x00\x00\x00\x00\x00\x8c\x88" -> /BYTES/ +CPut /Table/106/2/"B\xfd\x10\x01P\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ +CPut /Table/106/2/"B\xfd\x10\x03\xff\xff\xfc\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ +CPut /Table/106/2/"B\xfd\x10\x05\x00\x00\x00\x00\x00\x00\x00?\x91\xdfF\xa2R\x9d8?\x91\xdfF\xa2R\x9c\xb9?\xa1\xdfF\xa2R\x9d9?\xa1\xdfF\xa2R\x9dx\x8c\x88" -> /BYTES/ +CPut /Table/106/4/"B\xfd\x10\x00\x00\x00\x00\x00\x00\x01\x01?\xf0\x00\x00\x00\x00\x00\x00?\xf0\x00\x00\x00\x00\x00\x00\x8c\x88" -> /BYTES/ statement ok CREATE TABLE ltable( diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index index 2acb5310c1a2..f6b31eb5423a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index @@ -35,15 +35,15 @@ query T kvtrace INSERT INTO d VALUES(0, '{"a": "b"}') ---- CPut /Table/106/1/0/0 -> /TUPLE/ -InitPut /Table/106/2/"a"/"b"/0/0 -> /BYTES/ +CPut /Table/106/2/"a"/"b"/0/0 -> /BYTES/ # Make sure duplicate values don't get inserted. query T kvtrace INSERT INTO d VALUES(1, '[7,0,7]') ---- CPut /Table/106/1/1/0 -> /TUPLE/ -InitPut /Table/106/2/Arr/0/1/0 -> /BYTES/ -InitPut /Table/106/2/Arr/7/1/0 -> /BYTES/ +CPut /Table/106/2/Arr/0/1/0 -> /BYTES/ +CPut /Table/106/2/Arr/7/1/0 -> /BYTES/ # Make sure duplicate values don't get deleted either. query T kvtrace @@ -58,17 +58,17 @@ query T kvtrace INSERT INTO d VALUES(2, '[{"a": "b"}, 3, {"a": "b"}]') ---- CPut /Table/106/1/2/0 -> /TUPLE/ -InitPut /Table/106/2/Arr/3/2/0 -> /BYTES/ -InitPut /Table/106/2/Arr/"a"/"b"/2/0 -> /BYTES/ +CPut /Table/106/2/Arr/3/2/0 -> /BYTES/ +CPut /Table/106/2/Arr/"a"/"b"/2/0 -> /BYTES/ query T kvtrace INSERT INTO d VALUES(3, '[{"a": [0,1,0]}, 3, {"a": "b"}]') ---- CPut /Table/106/1/3/0 -> /TUPLE/ -InitPut /Table/106/2/Arr/3/3/0 -> /BYTES/ -InitPut /Table/106/2/Arr/"a"/"b"/3/0 -> /BYTES/ -InitPut /Table/106/2/Arr/"a"/Arr/0/3/0 -> /BYTES/ -InitPut /Table/106/2/Arr/"a"/Arr/1/3/0 -> /BYTES/ +CPut /Table/106/2/Arr/3/3/0 -> /BYTES/ +CPut /Table/106/2/Arr/"a"/"b"/3/0 -> /BYTES/ +CPut /Table/106/2/Arr/"a"/Arr/0/3/0 -> /BYTES/ +CPut /Table/106/2/Arr/"a"/Arr/1/3/0 -> /BYTES/ # Make sure that inserting NULL doesn't make an index entry. query T kvtrace @@ -82,7 +82,7 @@ UPDATE d SET b='[1]' WHERE a=4 ---- Scan /Table/106/1/4/0 lock Exclusive (Block, Unreplicated) Put /Table/106/1/4/0 -> /TUPLE/ -InitPut /Table/106/2/Arr/1/4/0 -> /BYTES/ +CPut /Table/106/2/Arr/1/4/0 -> /BYTES/ # Update back to null. query T kvtrace @@ -108,18 +108,18 @@ INSERT INTO e VALUES(0, ARRAY[7,0,0,1,NULL,10,0,1,7,NULL]) ---- Scan /Table/20/1/10{7-8} CPut /Table/107/1/0/0 -> /TUPLE/ -InitPut /Table/107/2/NULL/0/0 -> /BYTES/ -InitPut /Table/107/2/0/0/0 -> /BYTES/ -InitPut /Table/107/2/1/0/0 -> /BYTES/ -InitPut /Table/107/2/7/0/0 -> /BYTES/ -InitPut /Table/107/2/10/0/0 -> /BYTES/ +CPut /Table/107/2/NULL/0/0 -> /BYTES/ +CPut /Table/107/2/0/0/0 -> /BYTES/ +CPut /Table/107/2/1/0/0 -> /BYTES/ +CPut /Table/107/2/7/0/0 -> /BYTES/ +CPut /Table/107/2/10/0/0 -> /BYTES/ # Make sure that empty arrays emit a key. query T kvtrace INSERT INTO e VALUES(1, ARRAY[]) ---- CPut /Table/107/1/1/0 -> /TUPLE/ -InitPut /Table/107/2/[]/1/0 -> /BYTES/ +CPut /Table/107/2/[]/1/0 -> /BYTES/ # Make sure that NULL arrays do not emit any keys at all. query T kvtrace @@ -132,7 +132,7 @@ query T kvtrace INSERT INTO e VALUES(3, ARRAY[NULL]) ---- CPut /Table/107/1/3/0 -> /TUPLE/ -InitPut /Table/107/2/NULL/3/0 -> /BYTES/ +CPut /Table/107/2/NULL/3/0 -> /BYTES/ # Make sure that we're emitting the right plan for array index scans even if # a column is both forward and inverted indexed. @@ -202,16 +202,16 @@ INSERT INTO f VALUES(0, ARRAY[7,0,0,1.000,10,0,1,7,1.0,1.00]) ---- Scan /Table/20/1/10{8-9} CPut /Table/108/1/0/0 -> /TUPLE/ -InitPut /Table/108/2/0/0/0 -> /BYTES/ -InitPut /Table/108/2/1/0/0 -> /BYTES/ -InitPut /Table/108/2/7/0/0 -> /BYTES/ -InitPut /Table/108/2/1E+1/0/0 -> /BYTES/ +CPut /Table/108/2/0/0/0 -> /BYTES/ +CPut /Table/108/2/1/0/0 -> /BYTES/ +CPut /Table/108/2/7/0/0 -> /BYTES/ +CPut /Table/108/2/1E+1/0/0 -> /BYTES/ query T kvtrace INSERT INTO f VALUES(1, ARRAY[]) ---- CPut /Table/108/1/1/0 -> /TUPLE/ -InitPut /Table/108/2/[]/1/0 -> /BYTES/ +CPut /Table/108/2/[]/1/0 -> /BYTES/ query T kvtrace INSERT INTO f VALUES(2, NULL) @@ -227,7 +227,7 @@ UPDATE f SET b = ARRAY[0,15,7,10] WHERE a = 0 Scan /Table/108/1/0/0 lock Exclusive (Block, Unreplicated) Put /Table/108/1/0/0 -> /TUPLE/ Del /Table/108/2/1/0/0 -InitPut /Table/108/2/15/0/0 -> /BYTES/ +CPut /Table/108/2/15/0/0 -> /BYTES/ statement error pgcode XXUUU could not produce a query plan conforming to the FORCE_INVERTED_INDEX hint SELECT * from d@{FORCE_INVERTED_INDEX} @@ -3235,8 +3235,8 @@ INSERT INTO t VALUES (1.00, ARRAY[1,2]) ---- Scan /Table/20/1/109{-/PrefixEnd} CPut /Table/109/1/1/0 -> /TUPLE/1:1:Decimal/1.00/ -InitPut /Table/109/2/1/1/0 -> /BYTES/0x1503348964 -InitPut /Table/109/2/2/1/0 -> /BYTES/0x1503348964 +CPut /Table/109/2/1/1/0 -> /BYTES/0x1503348964 +CPut /Table/109/2/2/1/0 -> /BYTES/0x1503348964 statement ok CREATE TABLE geo_table( diff --git a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index_multi_column b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index_multi_column index 36e2db63f394..08fe4e65ca29 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/inverted_index_multi_column +++ b/pkg/sql/opt/exec/execbuilder/testdata/inverted_index_multi_column @@ -31,8 +31,8 @@ query T kvtrace INSERT INTO t VALUES (1, 333, 'foo', '{"a": "b"}'::json) ---- CPut /Table/106/1/1/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo/ -InitPut /Table/106/2/333/"a"/"b"/1/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/"a"/"b"/1/0 -> /BYTES/ +CPut /Table/106/2/333/"a"/"b"/1/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/"a"/"b"/1/0 -> /BYTES/ # This test shows an inverted index scan followed by a primary index scan to # retrieve all the table's columns. @@ -47,19 +47,19 @@ query T kvtrace INSERT INTO t VALUES (2, 333, 'foo', '[7, 0, 7]'::json) ---- CPut /Table/106/1/2/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo/ -InitPut /Table/106/2/333/Arr/0/2/0 -> /BYTES/ -InitPut /Table/106/2/333/Arr/7/2/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/Arr/0/2/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/Arr/7/2/0 -> /BYTES/ +CPut /Table/106/2/333/Arr/0/2/0 -> /BYTES/ +CPut /Table/106/2/333/Arr/7/2/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/Arr/0/2/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/Arr/7/2/0 -> /BYTES/ query T kvtrace INSERT INTO t VALUES (3, 333, 'foo', '[{"a": "b"}, 3, {"a": "b"}]'::json) ---- CPut /Table/106/1/3/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo/ -InitPut /Table/106/2/333/Arr/3/3/0 -> /BYTES/ -InitPut /Table/106/2/333/Arr/"a"/"b"/3/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/Arr/3/3/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/Arr/"a"/"b"/3/0 -> /BYTES/ +CPut /Table/106/2/333/Arr/3/3/0 -> /BYTES/ +CPut /Table/106/2/333/Arr/"a"/"b"/3/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/Arr/3/3/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/Arr/"a"/"b"/3/0 -> /BYTES/ # Don't delete duplicate values. query T kvtrace @@ -94,8 +94,8 @@ UPDATE t SET j = '[1]' WHERE k = 4 ---- Scan /Table/106/1/4/0 lock Exclusive (Block, Unreplicated) Put /Table/106/1/4/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo/ -InitPut /Table/106/2/333/Arr/1/4/0 -> /BYTES/ -InitPut /Table/106/3/333/"foo"/Arr/1/4/0 -> /BYTES/ +CPut /Table/106/2/333/Arr/1/4/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/Arr/1/4/0 -> /BYTES/ # Update back to NULL. query T kvtrace @@ -118,8 +118,8 @@ query T kvtrace INSERT INTO t VALUES (5, NULL, 'foo', '{"a": "b"}'::json) ---- CPut /Table/106/1/5/0 -> /TUPLE/3:3:Bytes/foo/ -InitPut /Table/106/2/NULL/"a"/"b"/5/0 -> /BYTES/ -InitPut /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/2/NULL/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -> /BYTES/ # Update away from NULL. query T kvtrace @@ -128,9 +128,9 @@ UPDATE t SET i = 333 WHERE k = 5 Scan /Table/106/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/106/1/5/0 -> /TUPLE/2:2:Int/333/1:3:Bytes/foo/ Del /Table/106/2/NULL/"a"/"b"/5/0 -InitPut /Table/106/2/333/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/2/333/"a"/"b"/5/0 -> /BYTES/ Del /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -InitPut /Table/106/3/333/"foo"/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/3/333/"foo"/"a"/"b"/5/0 -> /BYTES/ # Update back to NULL. query T kvtrace @@ -139,9 +139,9 @@ UPDATE t SET i = NULL WHERE k = 5 Scan /Table/106/1/5/0 lock Exclusive (Block, Unreplicated) Put /Table/106/1/5/0 -> /TUPLE/3:3:Bytes/foo/ Del /Table/106/2/333/"a"/"b"/5/0 -InitPut /Table/106/2/NULL/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/2/NULL/"a"/"b"/5/0 -> /BYTES/ Del /Table/106/3/333/"foo"/"a"/"b"/5/0 -InitPut /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -> /BYTES/ +CPut /Table/106/3/NULL/"foo"/"a"/"b"/5/0 -> /BYTES/ # Delete row with NULL non-inverted row. query T kvtrace diff --git a/pkg/sql/opt/exec/execbuilder/testdata/json b/pkg/sql/opt/exec/execbuilder/testdata/json index 7d1be813c119..affdc3f4f0a3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/json +++ b/pkg/sql/opt/exec/execbuilder/testdata/json @@ -253,15 +253,15 @@ INSERT INTO composite VALUES (1, '1.00'::JSONB), (2, '1'::JSONB), (3, '2'::JSONB ---- Scan /Table/20/1/10{8-9} CPut /Table/108/1/1/0 -> /TUPLE/ -InitPut /Table/108/2/"H*\x02\x00\x00\x89\x88" -> /BYTES/0x2f0f0c200000002000000403348964 +CPut /Table/108/2/"H*\x02\x00\x00\x89\x88" -> /BYTES/0x2f0f0c200000002000000403348964 CPut /Table/108/1/2/0 -> /TUPLE/ -InitPut /Table/108/2/"H*\x02\x00\x00\x8a\x88" -> /BYTES/ +CPut /Table/108/2/"H*\x02\x00\x00\x8a\x88" -> /BYTES/ CPut /Table/108/1/3/0 -> /TUPLE/ -InitPut /Table/108/2/"H*\x04\x00\x00\x8b\x88" -> /BYTES/ +CPut /Table/108/2/"H*\x04\x00\x00\x8b\x88" -> /BYTES/ CPut /Table/108/1/4/0 -> /TUPLE/ -InitPut /Table/108/2/"H*\x06\x00\x00\x8c\x88" -> /BYTES/0x2f0f0c20000000200000040334891e +CPut /Table/108/2/"H*\x06\x00\x00\x8c\x88" -> /BYTES/0x2f0f0c20000000200000040334891e CPut /Table/108/1/5/0 -> /TUPLE/ -InitPut /Table/108/2/"G\x12a\x00\x01\x00\x8d\x88" -> /BYTES/ +CPut /Table/108/2/"G\x12a\x00\x01\x00\x8d\x88" -> /BYTES/ query T kvtrace SELECT j FROM composite where j = '1.00'::JSONB diff --git a/pkg/sql/opt/exec/execbuilder/testdata/partial_index b/pkg/sql/opt/exec/execbuilder/testdata/partial_index index eaa0a1673c5b..685ea575d905 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/partial_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/partial_index @@ -223,24 +223,24 @@ query T kvtrace INSERT INTO t VALUES (5, 4, 'bar') ---- CPut /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/5/0 -> /BYTES/ +CPut /Table/112/2/4/5/0 -> /BYTES/ # Inserted row matches the first partial index. query T kvtrace INSERT INTO t VALUES (6, 11, 'bar') ---- CPut /Table/112/1/6/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/bar -InitPut /Table/112/2/11/6/0 -> /BYTES/ -InitPut /Table/112/3/11/6/0 -> /BYTES/ +CPut /Table/112/2/11/6/0 -> /BYTES/ +CPut /Table/112/3/11/6/0 -> /BYTES/ # Inserted row matches both partial indexes. query T kvtrace INSERT INTO t VALUES (12, 11, 'foo') ---- CPut /Table/112/1/12/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/foo -InitPut /Table/112/2/11/12/0 -> /BYTES/ -InitPut /Table/112/3/11/12/0 -> /BYTES/ -InitPut /Table/112/4/"foo"/12/0 -> /BYTES/ +CPut /Table/112/2/11/12/0 -> /BYTES/ +CPut /Table/112/3/11/12/0 -> /BYTES/ +CPut /Table/112/4/"foo"/12/0 -> /BYTES/ # Inserted row does not match partial index with predicate column that is not # indexed. @@ -255,7 +255,7 @@ query T kvtrace INSERT INTO u VALUES (2, 3, 11) ---- CPut /Table/113/1/2/0 -> /TUPLE/2:2:Int/3/1:3:Int/11 -InitPut /Table/113/2/3/2/0 -> /BYTES/ +CPut /Table/113/2/3/2/0 -> /BYTES/ # --------------------------------------------------------- # DELETE @@ -441,8 +441,8 @@ Del /Table/112/2/11/20/0 Del /Table/112/3/11/20/0 Del /Table/112/1/20/0 CPut /Table/112/1/21/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/bar -InitPut /Table/112/2/11/21/0 -> /BYTES/ -InitPut /Table/112/3/11/21/0 -> /BYTES/ +CPut /Table/112/2/11/21/0 -> /BYTES/ +CPut /Table/112/3/11/21/0 -> /BYTES/ # Update the primary key of a row that currently matches the first partial # index. Also update the row so that the row no longer matches the first partial @@ -455,8 +455,8 @@ Del /Table/112/2/11/21/0 Del /Table/112/3/11/21/0 Del /Table/112/1/21/0 CPut /Table/112/1/22/0 -> /TUPLE/2:2:Int/9/1:3:Bytes/foo -InitPut /Table/112/2/9/22/0 -> /BYTES/ -InitPut /Table/112/4/"foo"/22/0 -> /BYTES/ +CPut /Table/112/2/9/22/0 -> /BYTES/ +CPut /Table/112/4/"foo"/22/0 -> /BYTES/ # --------------------------------------------------------- # INSERT ON CONFLICT DO NOTHING @@ -475,7 +475,7 @@ INSERT INTO t VALUES (5, 4, 'bar') ON CONFLICT DO NOTHING ---- Scan /Table/112/1/5/0 CPut /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/5/0 -> /BYTES/ +CPut /Table/112/2/4/5/0 -> /BYTES/ # Insert a conflicting row that matches no partial index. query T kvtrace @@ -489,8 +489,8 @@ INSERT INTO t VALUES (6, 11, 'bar') ON CONFLICT DO NOTHING ---- Scan /Table/112/1/6/0 CPut /Table/112/1/6/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/bar -InitPut /Table/112/2/11/6/0 -> /BYTES/ -InitPut /Table/112/3/11/6/0 -> /BYTES/ +CPut /Table/112/2/11/6/0 -> /BYTES/ +CPut /Table/112/3/11/6/0 -> /BYTES/ # Insert a conflicting row that matches the first partial index. query T kvtrace @@ -504,9 +504,9 @@ INSERT INTO t VALUES (12, 11, 'foo') ON CONFLICT DO NOTHING ---- Scan /Table/112/1/12/0 CPut /Table/112/1/12/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/foo -InitPut /Table/112/2/11/12/0 -> /BYTES/ -InitPut /Table/112/3/11/12/0 -> /BYTES/ -InitPut /Table/112/4/"foo"/12/0 -> /BYTES/ +CPut /Table/112/2/11/12/0 -> /BYTES/ +CPut /Table/112/3/11/12/0 -> /BYTES/ +CPut /Table/112/4/"foo"/12/0 -> /BYTES/ # Insert a conflicting row that matches both partial indexes. query T kvtrace @@ -536,7 +536,7 @@ INSERT INTO u VALUES (2, 3, 11) ON CONFLICT DO NOTHING ---- Scan /Table/113/1/2/0 CPut /Table/113/1/2/0 -> /TUPLE/2:2:Int/3/1:3:Int/11 -InitPut /Table/113/2/3/2/0 -> /BYTES/ +CPut /Table/113/2/3/2/0 -> /BYTES/ # Insert a conflicting row that matches the partial index with predicate column # that is not indexed. @@ -562,7 +562,7 @@ INSERT INTO t VALUES (5, 4, 'bar') ON CONFLICT (a) DO UPDATE SET b = 3 ---- Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) CPut /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/5/0 -> /BYTES/ +CPut /Table/112/2/4/5/0 -> /BYTES/ # Insert a conflicting row that matches no partial indexes before or after # the update. @@ -624,8 +624,8 @@ INSERT INTO t VALUES (6, 11, 'baz') ON CONFLICT (a) DO UPDATE SET b = 3 ---- Scan /Table/112/1/6/0 lock Exclusive (Block, Unreplicated) CPut /Table/112/1/6/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/baz -InitPut /Table/112/2/11/6/0 -> /BYTES/ -InitPut /Table/112/3/11/6/0 -> /BYTES/ +CPut /Table/112/2/11/6/0 -> /BYTES/ +CPut /Table/112/3/11/6/0 -> /BYTES/ # Insert a non-conflicting row that does not match the partial index with # predicate column that is not indexed. @@ -650,7 +650,7 @@ INSERT INTO u VALUES (2, 3, 11) ON CONFLICT (k) DO UPDATE SET u = 5 ---- Scan /Table/113/1/2/0 lock Exclusive (Block, Unreplicated) CPut /Table/113/1/2/0 -> /TUPLE/2:2:Int/3/1:3:Int/11 -InitPut /Table/113/2/3/2/0 -> /BYTES/ +CPut /Table/113/2/3/2/0 -> /BYTES/ # Insert a conflicting row that matches the partial index with predicate column # that is not indexed. @@ -679,7 +679,7 @@ INSERT INTO t VALUES (5, 4, 'bar') ON CONFLICT (a) DO UPDATE SET a = 5 ---- Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) CPut /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/5/0 -> /BYTES/ +CPut /Table/112/2/4/5/0 -> /BYTES/ # Insert a conflicting row that matches no partial indexes before or after # the update. @@ -690,7 +690,7 @@ Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) Del /Table/112/2/4/5/0 Del /Table/112/1/5/0 CPut /Table/112/1/6/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/6/0 -> /BYTES/ +CPut /Table/112/2/4/6/0 -> /BYTES/ # Insert a conflicting row that currently does not match the second partial # index before the update, but does match after the update. @@ -701,8 +701,8 @@ Scan /Table/112/1/6/0 lock Exclusive (Block, Unreplicated) Del /Table/112/2/4/6/0 Del /Table/112/1/6/0 CPut /Table/112/1/7/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/foo -InitPut /Table/112/2/4/7/0 -> /BYTES/ -InitPut /Table/112/4/"foo"/7/0 -> /BYTES/ +CPut /Table/112/2/4/7/0 -> /BYTES/ +CPut /Table/112/4/"foo"/7/0 -> /BYTES/ # Insert a conflicting row that currently matches the second partial index # before the update. Update the row so that the row no longer matches the second @@ -715,8 +715,8 @@ Del /Table/112/2/4/7/0 Del /Table/112/4/"foo"/7/0 Del /Table/112/1/7/0 CPut /Table/112/1/8/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/foo -InitPut /Table/112/2/11/8/0 -> /BYTES/ -InitPut /Table/112/3/11/8/0 -> /BYTES/ +CPut /Table/112/2/11/8/0 -> /BYTES/ +CPut /Table/112/3/11/8/0 -> /BYTES/ # Insert a conflicting row that that matches the first partial index before and # after the update and the index entry changes. @@ -728,8 +728,8 @@ Del /Table/112/2/11/8/0 Del /Table/112/3/11/8/0 Del /Table/112/1/8/0 CPut /Table/112/1/9/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/foobar -InitPut /Table/112/2/11/9/0 -> /BYTES/ -InitPut /Table/112/3/11/9/0 -> /BYTES/ +CPut /Table/112/2/11/9/0 -> /BYTES/ +CPut /Table/112/3/11/9/0 -> /BYTES/ # --------------------------------------------------------- # UPSERT @@ -748,7 +748,7 @@ UPSERT INTO t VALUES (5, 4, 'bar') ---- Scan /Table/112/1/5/0 lock Exclusive (Block, Unreplicated) CPut /Table/112/1/5/0 -> /TUPLE/2:2:Int/4/1:3:Bytes/bar -InitPut /Table/112/2/4/5/0 -> /BYTES/ +CPut /Table/112/2/4/5/0 -> /BYTES/ # Upsert a conflicting row that matches no partial indexes before or after # the update. @@ -810,8 +810,8 @@ UPSERT INTO t VALUES (9, 11, 'baz') ---- Scan /Table/112/1/9/0 lock Exclusive (Block, Unreplicated) CPut /Table/112/1/9/0 -> /TUPLE/2:2:Int/11/1:3:Bytes/baz -InitPut /Table/112/2/11/9/0 -> /BYTES/ -InitPut /Table/112/3/11/9/0 -> /BYTES/ +CPut /Table/112/2/11/9/0 -> /BYTES/ +CPut /Table/112/3/11/9/0 -> /BYTES/ # Upsert a non-conflicting row that does not match the partial index with # predicate column that is not indexed. @@ -836,7 +836,7 @@ UPSERT INTO u VALUES (2, 3, 11) ---- Scan /Table/113/1/2/0 lock Exclusive (Block, Unreplicated) CPut /Table/113/1/2/0 -> /TUPLE/2:2:Int/3/1:3:Int/11 -InitPut /Table/113/2/3/2/0 -> /BYTES/ +CPut /Table/113/2/3/2/0 -> /BYTES/ # Upsert a conflicting row that matches the partial index with predicate column # that is not indexed. @@ -875,8 +875,8 @@ query T kvtrace INSERT INTO inv VALUES (1, '{"x": "y", "num": 1}', 'foo') ---- CPut /Table/107/1/1/0 -> /TUPLE/ -InitPut /Table/107/2/"num"/1/1/0 -> /BYTES/ -InitPut /Table/107/2/"x"/"y"/1/0 -> /BYTES/ +CPut /Table/107/2/"num"/1/1/0 -> /BYTES/ +CPut /Table/107/2/"x"/"y"/1/0 -> /BYTES/ query T kvtrace INSERT INTO inv VALUES (2, '{"x": "y", "num": 2}', 'baz') @@ -923,7 +923,7 @@ UPDATE inv SET b = '{"x": "y", "num": 3}' WHERE a = 1 Scan /Table/107/1/1/0 lock Exclusive (Block, Unreplicated) Put /Table/107/1/1/0 -> /TUPLE/ Del /Table/107/2/"num"/1/1/0 -InitPut /Table/107/2/"num"/3/1/0 -> /BYTES/ +CPut /Table/107/2/"num"/3/1/0 -> /BYTES/ # Update a non-JSON column so that the row is removed from the partial index. query T kvtrace @@ -954,8 +954,8 @@ UPDATE inv SET c = 'bar' WHERE a = 2 ---- Scan /Table/107/1/2/0 lock Exclusive (Block, Unreplicated) Put /Table/107/1/2/0 -> /TUPLE/ -InitPut /Table/107/2/"num"/4/2/0 -> /BYTES/ -InitPut /Table/107/2/"x"/"y"/2/0 -> /BYTES/ +CPut /Table/107/2/"num"/4/2/0 -> /BYTES/ +CPut /Table/107/2/"x"/"y"/2/0 -> /BYTES/ # Update the primary key of a row in the partial index. query T kvtrace @@ -966,8 +966,8 @@ Del /Table/107/2/"num"/4/2/0 Del /Table/107/2/"x"/"y"/2/0 Del /Table/107/1/2/0 CPut /Table/107/1/4/0 -> /TUPLE/ -InitPut /Table/107/2/"num"/4/4/0 -> /BYTES/ -InitPut /Table/107/2/"x"/"y"/4/0 -> /BYTES/ +CPut /Table/107/2/"num"/4/4/0 -> /BYTES/ +CPut /Table/107/2/"x"/"y"/4/0 -> /BYTES/ # Update the primary key of a row not in the partial index. query T kvtrace @@ -988,7 +988,7 @@ UPDATE inv SET c = 'foo' WHERE a IN (10, 11) Scan /Table/107/1/1{0-2} lock Exclusive (Block, Unreplicated) Put /Table/107/1/10/0 -> /TUPLE/ Put /Table/107/1/11/0 -> /TUPLE/ -InitPut /Table/107/2/"a"/"b"/11/0 -> /BYTES/ +CPut /Table/107/2/"a"/"b"/11/0 -> /BYTES/ # Update to multiple rows (one in and one not in the partial index) so that both # are not in the partial index. @@ -1023,7 +1023,7 @@ UPSERT INTO inv VALUES (4, '{"x": "y", "num": 6}', 'foo') Scan /Table/107/1/4/0 lock Exclusive (Block, Unreplicated) Put /Table/107/1/4/0 -> /TUPLE/ Del /Table/107/2/"num"/4/4/0 -InitPut /Table/107/2/"num"/6/4/0 -> /BYTES/ +CPut /Table/107/2/"num"/6/4/0 -> /BYTES/ # Upsert a conflicting row so that it is removed from the partial index. query T kvtrace @@ -1040,8 +1040,8 @@ UPSERT INTO inv VALUES (5, '{"x": "y", "num": 7}', 'bar') ---- Scan /Table/107/1/5/0 lock Exclusive (Block, Unreplicated) CPut /Table/107/1/5/0 -> /TUPLE/ -InitPut /Table/107/2/"num"/7/5/0 -> /BYTES/ -InitPut /Table/107/2/"x"/"y"/5/0 -> /BYTES/ +CPut /Table/107/2/"num"/7/5/0 -> /BYTES/ +CPut /Table/107/2/"x"/"y"/5/0 -> /BYTES/ # Upsert a non-conflicting row that is not added to the partial index. query T kvtrace diff --git a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families index 0076ccda9c31..5a9707457255 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families +++ b/pkg/sql/opt/exec/execbuilder/testdata/secondary_index_column_families @@ -15,17 +15,21 @@ CREATE TABLE t1 ( # be a single kv pair of the old format (BYTES value with PK cols in # the value, if needed). Inserts into nonuniqueidxstoring and # uniqueidxstoring both should generate 3 K/V pairs. -query T kvtrace(InitPut) +query T kvtrace(CPut) INSERT INTO t1 VALUES (1, 1, 1, 1, 1) ---- -InitPut /Table/106/2/1/1/0 -> /BYTES/ -InitPut /Table/106/3/1/0 -> /BYTES/0x89 -InitPut /Table/106/4/1/1/0 -> /BYTES/ -InitPut /Table/106/4/1/1/2/1 -> /TUPLE/3:3:Int/1 -InitPut /Table/106/4/1/1/3/1 -> /TUPLE/4:4:Int/1/1:5:Int/1 -InitPut /Table/106/5/1/0 -> /BYTES/0x89 -InitPut /Table/106/5/1/2/1 -> /TUPLE/3:3:Int/1 -InitPut /Table/106/5/1/3/1 -> /TUPLE/4:4:Int/1/1:5:Int/1 +CPut /Table/106/1/1/0 -> /TUPLE/ +CPut /Table/106/1/1/1/1 -> /INT/1 +CPut /Table/106/1/1/2/1 -> /INT/1 +CPut /Table/106/1/1/3/1 -> /TUPLE/4:4:Int/1/1:5:Int/1 +CPut /Table/106/2/1/1/0 -> /BYTES/ +CPut /Table/106/3/1/0 -> /BYTES/0x89 +CPut /Table/106/4/1/1/0 -> /BYTES/ +CPut /Table/106/4/1/1/2/1 -> /TUPLE/3:3:Int/1 +CPut /Table/106/4/1/1/3/1 -> /TUPLE/4:4:Int/1/1:5:Int/1 +CPut /Table/106/5/1/0 -> /BYTES/0x89 +CPut /Table/106/5/1/2/1 -> /TUPLE/3:3:Int/1 +CPut /Table/106/5/1/3/1 -> /TUPLE/4:4:Int/1/1:5:Int/1 # Deletes on nonuniqueidx or uniqueidx should result in a deletion @@ -147,43 +151,43 @@ SET TRACING=off; query T SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'Del /Table/106/2/%' OR -message LIKE 'InitPut /Table/106/2/%' OR +message LIKE '%Put /Table/106/2/%' OR message LIKE 'Del /Table/106/3/%' OR -message LIKE 'InitPut /Table/106/3/%' +message LIKE '%Put /Table/106/3/%' ORDER BY message ---- +CPut /Table/106/2/2/2/0 -> /BYTES/ +CPut /Table/106/3/2/0 -> /BYTES/0x8a Del /Table/106/2/1/1/0 Del /Table/106/3/1/0 -InitPut /Table/106/2/2/2/0 -> /BYTES/ -InitPut /Table/106/3/2/0 -> /BYTES/0x8a # Updates on nonuniqueidxstoring should generate 3 K/V pairs. query T SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'Del /Table/106/4/%' OR -message LIKE 'InitPut /Table/106/4/%' +message LIKE '%Put /Table/106/4/%' ORDER BY message ---- +CPut /Table/106/4/2/2/0 -> /BYTES/ +CPut /Table/106/4/2/2/2/1 -> /TUPLE/3:3:Int/2 +CPut /Table/106/4/2/2/3/1 -> /TUPLE/4:4:Int/2/1:5:Int/2 Del /Table/106/4/1/1/0 Del /Table/106/4/1/1/2/1 Del /Table/106/4/1/1/3/1 -InitPut /Table/106/4/2/2/0 -> /BYTES/ -InitPut /Table/106/4/2/2/2/1 -> /TUPLE/3:3:Int/2 -InitPut /Table/106/4/2/2/3/1 -> /TUPLE/4:4:Int/2/1:5:Int/2 # Updates on uniqueidxstoring should generate 3 K/V pairs. query T SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE 'Del /Table/106/5/%' OR -message LIKE 'InitPut /Table/106/5/%' +message LIKE '%Put /Table/106/5/%' ORDER BY message ---- +CPut /Table/106/5/2/0 -> /BYTES/0x8a +CPut /Table/106/5/2/2/1 -> /TUPLE/3:3:Int/2 +CPut /Table/106/5/2/3/1 -> /TUPLE/4:4:Int/2/1:5:Int/2 Del /Table/106/5/1/0 Del /Table/106/5/1/2/1 Del /Table/106/5/1/3/1 -InitPut /Table/106/5/2/0 -> /BYTES/0x8a -InitPut /Table/106/5/2/2/1 -> /TUPLE/3:3:Int/2 -InitPut /Table/106/5/2/3/1 -> /TUPLE/4:4:Int/2/1:5:Int/2 # Ensure that reads only scan the necessary k/v's. statement ok @@ -380,17 +384,17 @@ CREATE TABLE t ( ); # Ensure we only insert the correct keys. -query T kvtrace(InitPut,prefix=/Table/112/2/,prefix=/Table/112/3/) +query T kvtrace(CPut,prefix=/Table/112/2/,prefix=/Table/112/3/) INSERT INTO t VALUES (1, 2, 3, NULL, 5, 6, NULL, 8) ---- -InitPut /Table/112/2/2/1/0 -> /BYTES/ -InitPut /Table/112/2/2/1/2/1 -> /TUPLE/3:3:Int/3 -InitPut /Table/112/2/2/1/3/1 -> /TUPLE/5:5:Int/5/1:6:Int/6 -InitPut /Table/112/2/2/1/5/1 -> /TUPLE/8:8:Int/8 -InitPut /Table/112/3/2/0 -> /BYTES/0x89 -InitPut /Table/112/3/2/2/1 -> /TUPLE/3:3:Int/3 -InitPut /Table/112/3/2/3/1 -> /TUPLE/5:5:Int/5/1:6:Int/6 -InitPut /Table/112/3/2/5/1 -> /TUPLE/8:8:Int/8 +CPut /Table/112/2/2/1/0 -> /BYTES/ +CPut /Table/112/2/2/1/2/1 -> /TUPLE/3:3:Int/3 +CPut /Table/112/2/2/1/3/1 -> /TUPLE/5:5:Int/5/1:6:Int/6 +CPut /Table/112/2/2/1/5/1 -> /TUPLE/8:8:Int/8 +CPut /Table/112/3/2/0 -> /BYTES/0x89 +CPut /Table/112/3/2/2/1 -> /TUPLE/3:3:Int/3 +CPut /Table/112/3/2/3/1 -> /TUPLE/5:5:Int/5/1:6:Int/6 +CPut /Table/112/3/2/5/1 -> /TUPLE/8:8:Int/8 # Test some cases of the updater. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 4957f1c6c058..cfc4e388d934 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -90,7 +90,7 @@ query TT $trace_query ---- count CPut /Table/108/1/1/0 -> /TUPLE/2:2:Int/2 -count InitPut /Table/108/2/2/0 -> /BYTES/0x89 +count CPut /Table/108/2/2/0 -> /BYTES/0x89 count fast path completed sql query rows affected: 1 @@ -103,7 +103,7 @@ set tracing=off; $trace_query ---- count CPut /Table/108/1/1/0 -> /TUPLE/2:2:Int/2 -count InitPut /Table/108/2/2/0 -> /BYTES/0x89 +count CPut /Table/108/2/2/0 -> /BYTES/0x89 sql query execution failed after 0 rows: duplicate key value violates unique constraint "kv_pkey" statement error duplicate key value @@ -114,7 +114,7 @@ set tracing=off; $trace_query ---- count CPut /Table/108/1/2/0 -> /TUPLE/2:2:Int/2 -count InitPut /Table/108/2/2/0 -> /BYTES/0x8a +count CPut /Table/108/2/2/0 -> /BYTES/0x8a sql query execution failed after 0 rows: duplicate key value violates unique constraint "woo" statement ok diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 8248ac7aecab..1ef64f13e318 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -916,7 +916,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- colbatchscan Scan /Table/120/1/2/0 lock Exclusive (Block, Unreplicated) count CPut /Table/120/1/2/0 -> /TUPLE/2:2:Int/3 -count InitPut /Table/120/2/3/0 -> /BYTES/0x8a +count CPut /Table/120/2/3/0 -> /BYTES/0x8a count fast path completed sql query rows affected: 1 @@ -929,7 +929,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] ---- colbatchscan Scan /Table/120/1/1/0 lock Exclusive (Block, Unreplicated) count CPut /Table/120/1/1/0 -> /TUPLE/2:2:Int/2 -count InitPut /Table/120/2/2/0 -> /BYTES/0x89 +count CPut /Table/120/2/2/0 -> /BYTES/0x89 count fast path completed sql query rows affected: 1 diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index e9a5a557892a..fec7eaaace1e 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -110,9 +110,9 @@ func insertInvertedPutFn( ctx context.Context, b Putter, key *roachpb.Key, value *roachpb.Value, traceKV bool, ) { if traceKV { - log.VEventfDepth(ctx, 1, 2, "InitPut %s -> %s", *key, value.PrettyPrint()) + log.VEventfDepth(ctx, 1, 2, "CPut %s -> %s", *key, value.PrettyPrint()) } - b.InitPut(key, value, false) + b.CPut(key, value, nil /* expValue */) } func writeTombstones( diff --git a/pkg/sql/row/putter.go b/pkg/sql/row/putter.go index 2d41a75d83d6..439097eea7f5 100644 --- a/pkg/sql/row/putter.go +++ b/pkg/sql/row/putter.go @@ -22,15 +22,13 @@ type Putter interface { CPut(key, value interface{}, expValue []byte) CPutWithOriginTimestamp(key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool) Put(key, value interface{}) - InitPut(key, value interface{}, failOnTombstones bool) Del(key ...interface{}) + CPutBytesEmpty(kys []roachpb.Key, values [][]byte) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) CPutTuplesEmpty(kys []roachpb.Key, values [][]byte) PutBytes(kys []roachpb.Key, values [][]byte) - InitPutBytes(kys []roachpb.Key, values [][]byte) PutTuples(kys []roachpb.Key, values [][]byte) - InitPutTuples(kys []roachpb.Key, values [][]byte) } // TracePutter logs all requests, ie implements kv trace. @@ -57,16 +55,24 @@ func (t *TracePutter) Put(key, value interface{}) { log.VEventfDepth(t.Ctx, 1, 2, "Put %v -> %v", key, value) t.Putter.Put(key, value) } -func (t *TracePutter) InitPut(key, value interface{}, failOnTombstones bool) { - log.VEventfDepth(t.Ctx, 1, 2, "InitPut %v -> %v", key, value) - t.Putter.Put(key, value) -} func (t *TracePutter) Del(key ...interface{}) { log.VEventfDepth(t.Ctx, 1, 2, "Del %v", key...) t.Putter.Del(key...) } +func (t *TracePutter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) { + for i, k := range kys { + if len(k) == 0 { + continue + } + var v roachpb.Value + v.SetBytes(values[i]) + log.VEventfDepth(t.Ctx, 1, 2, "CPut %s -> %s", k, v.PrettyPrint()) + } + t.Putter.CPutBytesEmpty(kys, values) +} + func (t *TracePutter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) { for i, k := range kys { if len(k) == 0 { @@ -101,18 +107,6 @@ func (t *TracePutter) PutBytes(kys []roachpb.Key, values [][]byte) { t.Putter.PutBytes(kys, values) } -func (t *TracePutter) InitPutBytes(kys []roachpb.Key, values [][]byte) { - for i, k := range kys { - if len(k) == 0 { - continue - } - var v roachpb.Value - v.SetBytes(values[i]) - log.VEventfDepth(t.Ctx, 1, 2, "InitPut %s -> %s", k, v.PrettyPrint()) - } - t.Putter.InitPutBytes(kys, values) -} - func (t *TracePutter) PutTuples(kys []roachpb.Key, values [][]byte) { for i, k := range kys { if len(k) == 0 { @@ -125,18 +119,6 @@ func (t *TracePutter) PutTuples(kys []roachpb.Key, values [][]byte) { t.Putter.PutTuples(kys, values) } -func (t *TracePutter) InitPutTuples(kys []roachpb.Key, values [][]byte) { - for i, k := range kys { - if len(k) == 0 { - continue - } - var v roachpb.Value - v.SetTuple(values[i]) - log.VEventfDepth(t.Ctx, 1, 2, "InitPut %s -> %s", k, v.PrettyPrint()) - } - t.Putter.InitPutTuples(kys, values) -} - type KVBytes struct { Keys []roachpb.Key Values [][]byte @@ -196,14 +178,17 @@ func (s *SortingPutter) CPutWithOriginTimestamp( func (s *SortingPutter) Put(key, value interface{}) { s.Putter.Put(key, value) } -func (s *SortingPutter) InitPut(key, value interface{}, failOnTombstones bool) { - s.Putter.InitPut(key, value, failOnTombstones) -} func (s *SortingPutter) Del(key ...interface{}) { s.Putter.Del(key...) } +func (s *SortingPutter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) { + kvs := KVBytes{Keys: kys, Values: values} + sort.Sort(&kvs) + s.Putter.CPutBytesEmpty(kvs.Keys, kvs.Values) +} + func (s *SortingPutter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) { kvs := KVVals{Keys: kys, Values: values} sort.Sort(&kvs) @@ -222,24 +207,12 @@ func (s *SortingPutter) PutBytes(kys []roachpb.Key, values [][]byte) { s.Putter.PutBytes(kvs.Keys, kvs.Values) } -func (s *SortingPutter) InitPutBytes(kys []roachpb.Key, values [][]byte) { - kvs := KVBytes{Keys: kys, Values: values} - sort.Sort(&kvs) - s.Putter.InitPutBytes(kvs.Keys, kvs.Values) -} - func (s *SortingPutter) PutTuples(kys []roachpb.Key, values [][]byte) { kvs := KVBytes{Keys: kys, Values: values} sort.Sort(&kvs) s.Putter.PutTuples(kvs.Keys, kvs.Values) } -func (s *SortingPutter) InitPutTuples(kys []roachpb.Key, values [][]byte) { - kvs := KVBytes{Keys: kys, Values: values} - sort.Sort(&kvs) - s.Putter.InitPutTuples(kvs.Keys, kvs.Values) -} - type kvSparseSliceBulkSource[T kv.GValue] struct { keys []roachpb.Key values []T @@ -298,14 +271,15 @@ func (k *KVBatchAdapter) CPut(key, value interface{}, expValue []byte) { func (k *KVBatchAdapter) Put(key, value interface{}) { k.Batch.Put(key, value) } -func (k *KVBatchAdapter) InitPut(key, value interface{}, failOnTombstones bool) { - k.Batch.InitPut(key, value, failOnTombstones) -} func (k *KVBatchAdapter) Del(key ...interface{}) { k.Batch.Del(key...) } +func (k *KVBatchAdapter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) { + k.Batch.CPutBytesEmpty(&kvSparseSliceBulkSource[[]byte]{kys, values}) +} + func (k *KVBatchAdapter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) { k.Batch.CPutValuesEmpty(&kvSparseSliceBulkSource[roachpb.Value]{kys, values}) } @@ -318,14 +292,6 @@ func (k *KVBatchAdapter) PutBytes(kys []roachpb.Key, values [][]byte) { k.Batch.PutBytes(&kvSparseSliceBulkSource[[]byte]{kys, values}) } -func (k *KVBatchAdapter) InitPutBytes(kys []roachpb.Key, values [][]byte) { - k.Batch.InitPutBytes(&kvSparseSliceBulkSource[[]byte]{kys, values}) -} - func (k *KVBatchAdapter) PutTuples(kys []roachpb.Key, values [][]byte) { k.Batch.PutTuples(&kvSparseSliceBulkSource[[]byte]{kys, values}) } - -func (k *KVBatchAdapter) InitPutTuples(kys []roachpb.Key, values [][]byte) { - k.Batch.InitPutTuples(&kvSparseSliceBulkSource[[]byte]{kys, values}) -} diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 56e9f8fd019d..731654c57d01 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -29,12 +29,18 @@ import ( "github.com/cockroachdb/errors" ) -// KVInserter implements the putter interface. +// KVInserter implements the row.Putter interface. type KVInserter func(roachpb.KeyValue) -// CPut is not implmented. +// CPut implements the row.Putter interface. func (i KVInserter) CPut(key, value interface{}, expValue []byte) { - panic("unimplemented") + if expValue != nil { + panic(errors.AssertionFailedf("unexpected non-nil expValue in CPut in KVInserter: %v", expValue)) + } + i(roachpb.KeyValue{ + Key: *key.(*roachpb.Key), + Value: *value.(*roachpb.Value), + }) } // Del is not implemented. @@ -53,7 +59,7 @@ func (i KVInserter) Del(key ...interface{}) { // empty). } -// Put method of the putter interface. +// Put method of the row.Putter interface. func (i KVInserter) Put(key, value interface{}) { i(roachpb.KeyValue{ Key: *key.(*roachpb.Key), @@ -61,23 +67,15 @@ func (i KVInserter) Put(key, value interface{}) { }) } -// InitPut method of the putter interface. -func (i KVInserter) InitPut(key, value interface{}, failOnTombstones bool) { - i(roachpb.KeyValue{ - Key: *key.(*roachpb.Key), - Value: *value.(*roachpb.Value), - }) -} func (c KVInserter) CPutWithOriginTimestamp( key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool, ) { } +func (c KVInserter) CPutBytesEmpty(kys []roachpb.Key, values [][]byte) {} func (c KVInserter) CPutTuplesEmpty(kys []roachpb.Key, values [][]byte) {} func (c KVInserter) CPutValuesEmpty(kys []roachpb.Key, values []roachpb.Value) {} func (c KVInserter) PutBytes(kys []roachpb.Key, values [][]byte) {} -func (c KVInserter) InitPutBytes(kys []roachpb.Key, values [][]byte) {} func (c KVInserter) PutTuples(kys []roachpb.Key, values [][]byte) {} -func (c KVInserter) InitPutTuples(kys []roachpb.Key, values [][]byte) {} // GenerateInsertRow prepares a row tuple for insertion. It fills in default // expressions, verifies non-nullable columns, and checks column widths. diff --git a/pkg/sql/tests/sysbench_test.go b/pkg/sql/tests/sysbench_test.go index 32ca22c29db4..449700af2b1b 100644 --- a/pkg/sql/tests/sysbench_test.go +++ b/pkg/sql/tests/sysbench_test.go @@ -458,7 +458,7 @@ func (s *sysbenchKV) DeleteInsert(t tableNum, id rowID, newK kValue, _ cValue, _ var b2 kv.Batch b2.CPut(pkKey, pkValue, nil /* expValue */) - b2.InitPut(newIndexKey, newIndexValue, false /* failOnTombstones */) + b2.CPut(newIndexKey, newIndexValue, nil /* expValue */) try0(s.txn.Run(s.ctx, &b2)) } diff --git a/pkg/upgrade/upgrades/descriptor_utils.go b/pkg/upgrade/upgrades/descriptor_utils.go index 4337ddfbc195..9aaf5016e382 100644 --- a/pkg/upgrade/upgrades/descriptor_utils.go +++ b/pkg/upgrade/upgrades/descriptor_utils.go @@ -99,7 +99,7 @@ func CreateSystemTableInTxn( b.CPut(tKey, desc.GetID(), nil) b.CPut(catalogkeys.MakeDescMetadataKey(codec, desc.GetID()), desc.DescriptorProto(), nil) if desc.IsSequence() { - b.InitPut(codec.SequenceKey(uint32(desc.GetID())), desc.GetSequenceOpts().Start, false /* failOnTombstones */) + b.CPut(codec.SequenceKey(uint32(desc.GetID())), desc.GetSequenceOpts().Start, nil /* expValue */) } if err := txn.Run(ctx, b); err != nil { return descpb.InvalidID, false, err