Skip to content

Commit

Permalink
kvserver: don't use marshaled proto for liveness CPut
Browse files Browse the repository at this point in the history
Fixes #38308

Updates to the liveness records are done as CPuts. Before this patch,
the CPuts' expected value was the re-marshaled proto with the previous
version. As #38308 explains, that's a bad practice since it prevents the
proto's encoding to change in any way (e.g. fields can't be removed).
Instead, this patch moves to keeping track of the raw bytes that have
been read from the DB, besides the unmarshalled liveness protos. The raw
bytes are used as the expected values.

Release note: None
  • Loading branch information
andreimatei committed May 19, 2020
1 parent 41e568c commit 8d275dc
Show file tree
Hide file tree
Showing 11 changed files with 263 additions and 212 deletions.
37 changes: 2 additions & 35 deletions pkg/kv/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,25 +410,12 @@ func (b *Batch) PutInline(key, value interface{}) {
// and Result.Err will indicate success or failure.
//
// 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).
// protoutil.Message, a *roachpb.Value, or any Go primitive type (bool, int,
// etc).
func (b *Batch) CPut(key, value interface{}, expValue *roachpb.Value) {
b.cputInternal(key, value, expValue, false)
}

// CPutDeprecated conditionally sets the value for a key if the existing value is equal
// to expValue. To conditionally set a value only if there is no existing entry
// pass nil for expValue. Note that this must be an interface{}(nil), not a
// typed nil value (e.g. []byte(nil)).
//
// A new result will be appended to the batch which will contain a single row
// and Result.Err will indicate success or failure.
//
// 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).
func (b *Batch) CPutDeprecated(key, value, expValue interface{}) {
b.cputInternalDeprecated(key, value, expValue, false)
}

// CPutAllowingIfNotExists is like CPut except it also allows the Put when the
// existing entry does not exist -- i.e. it succeeds if there is no existing
// entry or the existing entry has the expected value.
Expand Down Expand Up @@ -459,26 +446,6 @@ func (b *Batch) cputInternal(key, value interface{}, expValue *roachpb.Value, al
b.initResult(1, 1, notRaw, nil)
}

func (b *Batch) cputInternalDeprecated(key, value, expValue interface{}, allowNotExist 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
}
ev, err := marshalValue(expValue)
if err != nil {
b.initResult(0, 1, notRaw, err)
return
}
b.appendReqs(roachpb.NewConditionalPut(k, v, ev, allowNotExist))
b.initResult(1, 1, notRaw, nil)
}

// 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
Expand Down
8 changes: 0 additions & 8 deletions pkg/kv/kvserver/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"reflect"
"testing"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/storagepb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
Expand Down Expand Up @@ -92,13 +91,6 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{
emptySum: 13621293256077144893,
populatedSum: 13375098491754757572,
},
reflect.TypeOf(&storagepb.Liveness{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
return storagepb.NewPopulatedLiveness(r, false)
},
emptySum: 892800390935990883,
populatedSum: 16231745342114354146,
},
// This is used downstream of Raft only to write it into unreplicated keyspace
// as part of VersionUnreplicatedRaftTruncatedState.
// However, it has been sent through Raft for a long time, as part of
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -500,13 +500,13 @@ func (r *Replica) ReadProtectedTimestamps(ctx context.Context) {
}

func (nl *NodeLiveness) SetDrainingInternal(
ctx context.Context, liveness storagepb.Liveness, drain bool,
ctx context.Context, liveness LivenessRecord, drain bool,
) error {
return nl.setDrainingInternal(ctx, liveness, drain, nil /* reporter */)
}

func (nl *NodeLiveness) SetDecommissioningInternal(
ctx context.Context, nodeID roachpb.NodeID, liveness storagepb.Liveness, decommission bool,
ctx context.Context, nodeID roachpb.NodeID, liveness LivenessRecord, decommission bool,
) (changeCommitted bool, err error) {
return nl.setDecommissioningInternal(ctx, nodeID, liveness, decommission)
}
Expand Down
Loading

0 comments on commit 8d275dc

Please sign in to comment.