Skip to content

Commit

Permalink
Merge #138707
Browse files Browse the repository at this point in the history
138707: sql,kv: replace InitPuts with CPuts r=yuzefovich a=yuzefovich

Addresses: #71074.
Epic: None

**kv: remove client side support of FailOnTombstones option of InitPut**

We recently removed the only usage of `FailOnTombstones` option of the InitPut, so we can remove the client side support for it completely now.

**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.

**kv: remove client side support of InitPut requests**

It was now only used it tests, so we remove the client side support of InitPut requests altogether. The server side support can be removed once the compatibility with 24.3 is no longer needed.

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed Jan 10, 2025
2 parents 88a839a + 46787b3 commit 03250fb
Show file tree
Hide file tree
Showing 37 changed files with 255 additions and 613 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -405,6 +405,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 @@ -414,6 +415,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 @@ -423,6 +425,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 @@ -432,6 +435,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 @@ -441,6 +445,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 @@ -451,6 +456,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
88 changes: 22 additions & 66 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ type Batch struct {
reqs []kvpb.RequestUnion

// approxMutationReqBytes tracks the approximate size of keys and values in
// mutations added to this batch via Put, CPut, InitPut, Del, etc.
// mutations added to this batch via Put, CPut, Del, etc.
approxMutationReqBytes int
// Set when AddRawRequest is used, in which case using the "other"
// operations renders the batch unusable.
Expand Down Expand Up @@ -88,8 +88,8 @@ type BulkSourceIterator[T GValue] interface {
}

// ApproximateMutationBytes returns the approximate byte size of the mutations
// added to this batch via Put, CPut, InitPut, Del, etc methods. Mutations added
// via AddRawRequest are not tracked.
// added to this batch via Put, CPut, Del, etc methods. Mutations added via
// AddRawRequest are not tracked.
func (b *Batch) ApproximateMutationBytes() int {
return b.approxMutationReqBytes
}
Expand Down Expand Up @@ -625,10 +625,10 @@ func (b *Batch) cputInternal(
b.initResult(1, 1, notRaw, nil)
}

// CPutTuplesEmpty allows multiple CPut tuple requests to be added to the batch
// as tuples using the BulkSource interface. The values for these keys are
// 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) CPutTuplesEmpty(bs BulkSource[[]byte]) {
func (b *Batch) CPutBytesEmpty(bs BulkSource[[]byte]) {
numKeys := bs.Len()
reqs := make([]struct {
req kvpb.ConditionalPutRequest
Expand All @@ -645,16 +645,16 @@ func (b *Batch) CPutTuplesEmpty(bs BulkSource[[]byte]) {
i++
k, v := bsi.Next()
pr.Key = k
pr.Value.SetTuple(v)
pr.Value.SetBytes(v)
pr.Value.InitChecksum(k)
return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes)
})
}

// CPutValuesEmpty allows multiple CPut tuple requests to be added to the batch
// as roachpb.Values using the BulkSource interface. The values for these keys
// are expected to be empty.
func (b *Batch) CPutValuesEmpty(bs BulkSource[roachpb.Value]) {
// 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.
func (b *Batch) CPutTuplesEmpty(bs BulkSource[[]byte]) {
numKeys := bs.Len()
reqs := make([]struct {
req kvpb.ConditionalPutRequest
Expand All @@ -671,77 +671,33 @@ func (b *Batch) CPutValuesEmpty(bs BulkSource[roachpb.Value]) {
i++
k, v := bsi.Next()
pr.Key = k
pr.Value = v
pr.Value.InitChecksum(k)
return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes)
})
}

// InitPut sets the first value for a key to value. An ConditionFailedError is
// reported if a value already exists for the key and it's not equal to the
// value passed in. If failOnTombstones is set to true, tombstones will return
// a ConditionFailedError just like a mismatched value.
//
// key can be either a byte slice or a string. value can be any key type, a
// protoutil.Message or any Go primitive type (bool, int, etc). It is illegal
// to set value to nil.
func (b *Batch) InitPut(key, value interface{}, failOnTombstones bool) {
k, err := marshalKey(key)
if err != nil {
b.initResult(0, 1, notRaw, err)
return
}
v, err := marshalValue(value)
if err != nil {
b.initResult(0, 1, notRaw, err)
return
}
b.appendReqs(kvpb.NewInitPut(k, v, failOnTombstones))
b.approxMutationReqBytes += len(k) + len(v.RawBytes)
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.SetTuple(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]) {
// CPutValuesEmpty allows multiple CPut tuple requests to be added to the batch
// as roachpb.Values using the BulkSource interface. The values for these keys
// are expected to be empty.
func (b *Batch) CPutValuesEmpty(bs BulkSource[roachpb.Value]) {
numKeys := bs.Len()
reqs := make([]struct {
req kvpb.InitPutRequest
union kvpb.RequestUnion_InitPut
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.InitPut = pr
union.ConditionalPut = pr
pr.AllowIfDoesNotExist = false
pr.ExpBytes = nil
i++
k, v := bsi.Next()
pr.Key = k
pr.Value.SetTuple(v)
pr.Value = v
pr.Value.InitChecksum(k)
return kvpb.RequestUnion{Value: union}, len(k) + len(pr.Value.RawBytes)
})
Expand Down
14 changes: 0 additions & 14 deletions pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -474,20 +474,6 @@ func (db *DB) CPutInline(ctx context.Context, key, value interface{}, expValue [
return getOneErr(db.Run(ctx, b), b)
}

// InitPut sets the first value for a key to value. A ConditionFailedError is
// reported if a value already exists for the key and it's not equal to the
// value passed in. If failOnTombstones is set to true, tombstones count as
// mismatched values and will cause a ConditionFailedError.
//
// key can be either a byte slice or a string. value can be any key type, a
// protoutil.Message or any Go primitive type (bool, int, etc). It is illegal to
// set value to nil.
func (db *DB) InitPut(ctx context.Context, key, value interface{}, failOnTombstones bool) error {
b := &Batch{}
b.InitPut(key, value, failOnTombstones)
return getOneErr(db.Run(ctx, b), b)
}

// 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
35 changes: 1 addition & 34 deletions pkg/kv/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -268,38 +268,6 @@ func TestDB_CPutInline(t *testing.T) {
}
}

func TestDB_InitPut(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
s, db := setup(t)
defer s.Stopper().Stop(context.Background())
ctx := context.Background()

if err := db.InitPut(ctx, "aa", "1", false); err != nil {
t.Fatal(err)
}
if err := db.InitPut(ctx, "aa", "1", false); err != nil {
t.Fatal(err)
}
if err := db.InitPut(ctx, "aa", "2", false); err == nil {
t.Fatal("expected error from init put")
}
if _, err := db.Del(ctx, "aa"); err != nil {
t.Fatal(err)
}
if err := db.InitPut(ctx, "aa", "2", true); err == nil {
t.Fatal("expected error from init put")
}
if err := db.InitPut(ctx, "aa", "1", false); err != nil {
t.Fatal(err)
}
result, err := db.Get(ctx, "aa")
if err != nil {
t.Fatal(err)
}
checkResult(t, []byte("1"), result.ValueBytes())
}

func TestDB_Inc(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down Expand Up @@ -1138,8 +1106,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
Loading

0 comments on commit 03250fb

Please sign in to comment.