Skip to content

Commit 00bc771

Browse files
committed
kv: remove ShouldWinOriginTimestampTie option from cmd cput
The current KV writer does not use ShouldWinOriginTimestampTie, so it can be removed as unused code. The KV LDR writer stopped using this approach for tie-breaking due to two issues: 1. When replaying replicated writes, they win LWW against themselves, leading to duplicate KVs. This is inefficient, slowing performance and wasting storage until garbage collection (GC) removes the duplicates. 2. The approach fails to handle semantics correctly in three-way replication scenarios. Release note: none Epic: CRDB-48647
1 parent 879f32a commit 00bc771

File tree

9 files changed

+17
-36
lines changed

9 files changed

+17
-36
lines changed

pkg/kv/batch.go

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -578,9 +578,7 @@ func (b *Batch) CPutAllowingIfNotExists(key, value interface{}, expValue []byte)
578578
// This is used by logical data replication and other uses of this API
579579
// are discouraged since the semantics are subject to change as
580580
// required by that feature.
581-
func (b *Batch) CPutWithOriginTimestamp(
582-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
583-
) {
581+
func (b *Batch) CPutWithOriginTimestamp(key, value interface{}, expValue []byte, ts hlc.Timestamp) {
584582
k, err := marshalKey(key)
585583
if err != nil {
586584
b.initResult(0, 1, notRaw, err)
@@ -594,7 +592,6 @@ func (b *Batch) CPutWithOriginTimestamp(
594592
}
595593
r := kvpb.NewConditionalPut(k, v, expValue, false)
596594
r.(*kvpb.ConditionalPutRequest).OriginTimestamp = ts
597-
r.(*kvpb.ConditionalPutRequest).ShouldWinOriginTimestampTie = shouldWinTie
598595
b.appendReqs(r)
599596
b.approxMutationReqBytes += len(k) + len(v.RawBytes)
600597
b.initResult(1, 1, notRaw, nil)

pkg/kv/kvpb/api.proto

Lines changed: 1 addition & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -368,13 +368,7 @@ message ConditionalPutRequest {
368368
// Used by logical data replication.
369369
util.hlc.Timestamp origin_timestamp = 8 [(gogoproto.nullable) = false];
370370

371-
// ShouldWinOriginTimestampTie, if true, indicates that if the "comparison
372-
// timestamp" (see comment on OriginTimestamp) and OriginTimestamp are equal,
373-
// then the ConditionalPut should succeed (assuming the expected and actual
374-
// bytes also match).
375-
//
376-
// This must only be used in conjunction with OriginTimestamp.
377-
bool should_win_origin_timestamp_tie = 9;
371+
reserved 9;
378372
}
379373

380374
// A ConditionalPutResponse is the return value from the

pkg/kv/kvserver/batcheval/cmd_conditional_put.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -79,9 +79,8 @@ func ConditionalPut(
7979
TargetLockConflictBytes: storage.TargetBytesPerLockConflictError.Get(&cArgs.EvalCtx.ClusterSettings().SV),
8080
Category: fs.BatchEvalReadCategory,
8181
},
82-
AllowIfDoesNotExist: storage.CPutMissingBehavior(args.AllowIfDoesNotExist),
83-
OriginTimestamp: args.OriginTimestamp,
84-
ShouldWinOriginTimestampTie: args.ShouldWinOriginTimestampTie,
82+
AllowIfDoesNotExist: storage.CPutMissingBehavior(args.AllowIfDoesNotExist),
83+
OriginTimestamp: args.OriginTimestamp,
8584
}
8685

8786
var err error

pkg/sql/colenc/bench_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -138,7 +138,7 @@ type noopPutter struct{}
138138

139139
func (n *noopPutter) CPut(key, value interface{}, expValue []byte) {}
140140
func (n *noopPutter) CPutWithOriginTimestamp(
141-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
141+
key, value interface{}, expValue []byte, ts hlc.Timestamp,
142142
) {
143143
}
144144
func (n *noopPutter) Put(key, value interface{}) {}

pkg/sql/colenc/encode_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -709,7 +709,7 @@ func (c *capturePutter) CPut(key, value interface{}, expValue []byte) {
709709
}
710710

711711
func (c *capturePutter) CPutWithOriginTimestamp(
712-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
712+
key, value interface{}, expValue []byte, ts hlc.Timestamp,
713713
) {
714714
colexecerror.InternalError(errors.New("unimplemented"))
715715
}

pkg/sql/row/helper.go

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -528,7 +528,6 @@ func (rh *RowHelper) deleteIndexEntry(
528528
// option set.
529529
type OriginTimestampCPutHelper struct {
530530
OriginTimestamp hlc.Timestamp
531-
ShouldWinTie bool
532531
// PreviousWasDeleted is used to indicate that the expected
533532
// value is non-existent. This is helpful in Deleter to
534533
// distinguish between a delete of a value that had no columns
@@ -551,7 +550,7 @@ func (oh *OriginTimestampCPutHelper) CPutFn(
551550
if traceKV {
552551
log.VEventfDepth(ctx, 1, 2, "CPutWithOriginTimestamp %s -> %s @ %s", *key, value.PrettyPrint(), oh.OriginTimestamp)
553552
}
554-
b.CPutWithOriginTimestamp(key, value, expVal, oh.OriginTimestamp, oh.ShouldWinTie)
553+
b.CPutWithOriginTimestamp(key, value, expVal, oh.OriginTimestamp)
555554
}
556555

557556
func (oh *OriginTimestampCPutHelper) DelWithCPut(
@@ -560,7 +559,7 @@ func (oh *OriginTimestampCPutHelper) DelWithCPut(
560559
if traceKV {
561560
log.VEventfDepth(ctx, 1, 2, "CPutWithOriginTimestamp %s -> nil (delete) @ %s", key, oh.OriginTimestamp)
562561
}
563-
b.CPutWithOriginTimestamp(key, nil, expVal, oh.OriginTimestamp, oh.ShouldWinTie)
562+
b.CPutWithOriginTimestamp(key, nil, expVal, oh.OriginTimestamp)
564563
}
565564

566565
func FetchSpecRequiresRawMVCCValues(spec fetchpb.IndexFetchSpec) bool {

pkg/sql/row/putter.go

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ import (
2020
// encoding logic to kv.Batch.
2121
type Putter interface {
2222
CPut(key, value interface{}, expValue []byte)
23-
CPutWithOriginTimestamp(key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool)
23+
CPutWithOriginTimestamp(key, value interface{}, expValue []byte, ts hlc.Timestamp)
2424
Put(key, value interface{})
2525
PutMustAcquireExclusiveLock(key, value interface{})
2626
Del(key ...interface{})
@@ -47,10 +47,10 @@ func (t *TracePutter) CPut(key, value interface{}, expValue []byte) {
4747
}
4848

4949
func (t *TracePutter) CPutWithOriginTimestamp(
50-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
50+
key, value interface{}, expValue []byte, ts hlc.Timestamp,
5151
) {
5252
log.VEventfDepth(t.Ctx, 1, 2, "CPutWithOriginTimestamp %v -> %v @ %v", key, value, ts)
53-
t.Putter.CPutWithOriginTimestamp(key, value, expValue, ts, shouldWinTie)
53+
t.Putter.CPutWithOriginTimestamp(key, value, expValue, ts)
5454
}
5555

5656
func (t *TracePutter) Put(key, value interface{}) {
@@ -182,9 +182,9 @@ func (s *SortingPutter) CPut(key, value interface{}, expValue []byte) {
182182
}
183183

184184
func (s *SortingPutter) CPutWithOriginTimestamp(
185-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
185+
key, value interface{}, expValue []byte, ts hlc.Timestamp,
186186
) {
187-
s.Putter.CPutWithOriginTimestamp(key, value, expValue, ts, shouldWinTie)
187+
s.Putter.CPutWithOriginTimestamp(key, value, expValue, ts)
188188
}
189189

190190
func (s *SortingPutter) Put(key, value interface{}) {
@@ -279,9 +279,9 @@ type KVBatchAdapter struct {
279279
var _ Putter = &KVBatchAdapter{}
280280

281281
func (k *KVBatchAdapter) CPutWithOriginTimestamp(
282-
key, value interface{}, expValue []byte, originTimestamp hlc.Timestamp, shouldWinTie bool,
282+
key, value interface{}, expValue []byte, originTimestamp hlc.Timestamp,
283283
) {
284-
k.Batch.CPutWithOriginTimestamp(key, value, expValue, originTimestamp, shouldWinTie)
284+
k.Batch.CPutWithOriginTimestamp(key, value, expValue, originTimestamp)
285285
}
286286

287287
func (k *KVBatchAdapter) CPut(key, value interface{}, expValue []byte) {

pkg/sql/row/row_converter.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ func (c KVInserter) PutMustAcquireExclusiveLock(key, value interface{}) {
7676
panic(errors.AssertionFailedf("unimplemented"))
7777
}
7878
func (c KVInserter) CPutWithOriginTimestamp(
79-
key, value interface{}, expValue []byte, ts hlc.Timestamp, shouldWinTie bool,
79+
key, value interface{}, expValue []byte, ts hlc.Timestamp,
8080
) {
8181
panic(errors.AssertionFailedf("unimplemented"))
8282
}

pkg/storage/mvcc.go

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -2897,13 +2897,6 @@ type ConditionalPutWriteOptions struct {
28972897
// See the comment on the OriginTimestamp field of
28982898
// kvpb.ConditionalPutRequest for more details.
28992899
OriginTimestamp hlc.Timestamp
2900-
// ShouldWinOriginTimestampTie indicates whether the value should be
2901-
// accepted if the origin timestamp is the same as the
2902-
// origin_timestamp/mvcc_timestamp of the existing value.
2903-
//
2904-
// See the comment on the ShouldWinOriginTimestampTie field of
2905-
// kvpb.ConditionalPutRequest for more details.
2906-
ShouldWinOriginTimestampTie bool
29072900
}
29082901

29092902
// MVCCConditionalPut sets the value for a specified key only if the expected
@@ -3033,8 +3026,7 @@ func mvccConditionalPutUsingIter(
30333026
}
30343027
} else {
30353028
valueFn = func(existVal optionalValue) (roachpb.Value, error) {
3036-
originTSWinner, existTS := existVal.isOriginTimestampWinner(opts.OriginTimestamp,
3037-
opts.ShouldWinOriginTimestampTie)
3029+
originTSWinner, existTS := existVal.isOriginTimestampWinner(opts.OriginTimestamp, false)
30383030
if !originTSWinner {
30393031
return roachpb.Value{}, &kvpb.ConditionFailedError{
30403032
OriginTimestampOlderThan: existTS,

0 commit comments

Comments
 (0)