Skip to content

Commit

Permalink
sql: replace InitPut with CPut
Browse files Browse the repository at this point in the history
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
  • Loading branch information
yuzefovich committed Dec 23, 2024
1 parent 8d61a42 commit 45bd97c
Show file tree
Hide file tree
Showing 25 changed files with 251 additions and 313 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand All @@ -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)
Expand Down
72 changes: 26 additions & 46 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down
8 changes: 6 additions & 2 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/ingesting/write_descs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/catalog/tabledesc/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand All @@ -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
Expand Down
4 changes: 1 addition & 3 deletions pkg/sql/colenc/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {}
6 changes: 3 additions & 3 deletions pkg/sql/colenc/encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down Expand Up @@ -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
Expand Down
41 changes: 12 additions & 29 deletions pkg/sql/colenc/encode_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colenc/inverted.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}

Expand Down
18 changes: 9 additions & 9 deletions pkg/sql/copy/testdata/copy_from
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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');
Expand Down Expand Up @@ -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))
Expand All @@ -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);
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
2 changes: 1 addition & 1 deletion pkg/sql/copy_from.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
24 changes: 12 additions & 12 deletions pkg/sql/opt/exec/execbuilder/testdata/geospatial
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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(
Expand Down
Loading

0 comments on commit 45bd97c

Please sign in to comment.