-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
cmd_end_transaction.go
1313 lines (1229 loc) · 56.9 KB
/
cmd_end_transaction.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2014 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package batcheval
import (
"bytes"
"context"
"fmt"
"math"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/abortspan"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
func init() {
RegisterReadWriteCommand(roachpb.EndTxn, declareKeysEndTxn, EndTxn)
}
// declareKeysWriteTransaction is the shared portion of
// declareKeys{End,Heartbeat}Transaction.
func declareKeysWriteTransaction(
_ ImmutableRangeState, header *roachpb.Header, req roachpb.Request, latchSpans *spanset.SpanSet,
) {
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.TransactionKey(req.Header().Key, header.Txn.ID),
})
}
}
func declareKeysEndTxn(
rs ImmutableRangeState,
header *roachpb.Header,
req roachpb.Request,
latchSpans, _ *spanset.SpanSet,
_ time.Duration,
) {
et := req.(*roachpb.EndTxnRequest)
declareKeysWriteTransaction(rs, header, req, latchSpans)
var minTxnTS hlc.Timestamp
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
minTxnTS = header.Txn.MinTimestamp
abortSpanAccess := spanset.SpanReadOnly
if !et.Commit {
// Rollback EndTxn requests may write to the abort span, either if
// their Poison flag is set, in which case they will add an abort
// span entry, or if their Poison flag is not set and an abort span
// entry already exists on this Range, in which case they will clear
// that entry.
abortSpanAccess = spanset.SpanReadWrite
}
latchSpans.AddNonMVCC(abortSpanAccess, roachpb.Span{
Key: keys.AbortSpanKey(rs.GetRangeID(), header.Txn.ID),
})
}
// If the request is intending to finalize the transaction record then it
// needs to declare a few extra keys.
if !et.IsParallelCommit() {
// All requests that intend on resolving local locks need to depend on
// the range descriptor because they need to determine which locks are
// within the local range.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.RangeDescriptorKey(rs.GetStartKey()),
})
// The spans may extend beyond this Range, but it's ok for the
// purpose of acquiring latches. The parts in our Range will
// be resolved eagerly.
for _, span := range et.LockSpans {
latchSpans.AddMVCC(spanset.SpanReadWrite, span, minTxnTS)
}
if et.InternalCommitTrigger != nil {
if st := et.InternalCommitTrigger.SplitTrigger; st != nil {
// Splits may read from the entire pre-split range (they read
// from the LHS in all cases, and the RHS only when the existing
// stats contain estimates). Splits declare non-MVCC read access
// across the entire LHS to block all concurrent writes to the
// LHS because their stat deltas will interfere with the
// non-delta stats computed as a part of the split. Splits
// declare non-MVCC write access across the entire RHS to block
// all concurrent reads and writes to the RHS because they will
// fail if applied after the split. (see
// https://github.com/cockroachdb/cockroach/issues/14881)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: st.LeftDesc.StartKey.AsRawKey(),
EndKey: st.LeftDesc.EndKey.AsRawKey(),
})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: st.RightDesc.StartKey.AsRawKey(),
EndKey: st.RightDesc.EndKey.AsRawKey(),
})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.MakeRangeKeyPrefix(st.LeftDesc.StartKey),
EndKey: keys.MakeRangeKeyPrefix(st.RightDesc.EndKey).PrefixEnd(),
})
leftRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(rs.GetRangeID())
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: leftRangeIDPrefix,
EndKey: leftRangeIDPrefix.PrefixEnd(),
})
rightRangeIDPrefix := keys.MakeRangeIDReplicatedPrefix(st.RightDesc.RangeID)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: rightRangeIDPrefix,
EndKey: rightRangeIDPrefix.PrefixEnd(),
})
rightRangeIDUnreplicatedPrefix := keys.MakeRangeIDUnreplicatedPrefix(st.RightDesc.RangeID)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: rightRangeIDUnreplicatedPrefix,
EndKey: rightRangeIDUnreplicatedPrefix.PrefixEnd(),
})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.RangeLastReplicaGCTimestampKey(st.LeftDesc.RangeID),
})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.RangeLastReplicaGCTimestampKey(st.RightDesc.RangeID),
})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: abortspan.MinKey(rs.GetRangeID()),
EndKey: abortspan.MaxKey(rs.GetRangeID()),
})
}
if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil {
// Merges copy over the RHS abort span to the LHS, and compute
// replicated range ID stats over the RHS in the merge trigger.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: abortspan.MinKey(mt.LeftDesc.RangeID),
EndKey: abortspan.MaxKey(mt.LeftDesc.RangeID).PrefixEnd(),
})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID),
EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(),
})
// Merges incorporate the prior read summary from the RHS into
// the LHS, which ensures that the current and all future
// leaseholders on the joint range respect reads served on the
// RHS.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.RangePriorReadSummaryKey(mt.LeftDesc.RangeID),
})
// Merges need to adjust MVCC stats for merged MVCC range tombstones
// that straddle the ranges, by peeking to the left and right of the RHS
// start key. Since Prevish() is imprecise, we must also ensure we don't
// go outside of the LHS bounds.
leftPeekBound := mt.RightDesc.StartKey.AsRawKey().Prevish(roachpb.PrevishKeyLength)
rightPeekBound := mt.RightDesc.StartKey.AsRawKey().Next()
if leftPeekBound.Compare(mt.LeftDesc.StartKey.AsRawKey()) < 0 {
leftPeekBound = mt.LeftDesc.StartKey.AsRawKey()
}
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: leftPeekBound,
EndKey: rightPeekBound,
})
}
}
}
}
// EndTxn either commits or aborts (rolls back) an extant transaction according
// to the args.Commit parameter. Rolling back an already rolled-back txn is ok.
// TODO(nvanbenschoten): rename this file to cmd_end_txn.go once some of andrei's
// recent PRs have landed.
func EndTxn(
ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, resp roachpb.Response,
) (result.Result, error) {
args := cArgs.Args.(*roachpb.EndTxnRequest)
h := cArgs.Header
ms := cArgs.Stats
reply := resp.(*roachpb.EndTxnResponse)
if err := VerifyTransaction(h, args, roachpb.PENDING, roachpb.STAGING, roachpb.ABORTED); err != nil {
return result.Result{}, err
}
if args.Require1PC {
// If a 1PC txn was required and we're in EndTxn, we've failed to evaluate
// the batch as a 1PC. We shouldn't have gotten here; if we couldn't
// evaluate as 1PC, evaluateWriteBatch() was supposed to short-circuit.
return result.Result{}, errors.AssertionFailedf("unexpectedly trying to evaluate EndTxn with the Require1PC flag set")
}
if args.Commit && args.Poison {
return result.Result{}, errors.AssertionFailedf("cannot poison during a committing EndTxn request")
}
key := keys.TransactionKey(h.Txn.Key, h.Txn.ID)
// Fetch existing transaction.
var existingTxn roachpb.Transaction
recordAlreadyExisted, err := storage.MVCCGetProto(
ctx, readWriter, key, hlc.Timestamp{}, &existingTxn, storage.MVCCGetOptions{},
)
if err != nil {
return result.Result{}, err
} else if !recordAlreadyExisted {
// No existing transaction record was found - create one by writing it
// below in updateFinalizedTxn.
reply.Txn = h.Txn.Clone()
// Verify that it is safe to create the transaction record. We only need
// to perform this verification for commits. Rollbacks can always write
// an aborted txn record.
if args.Commit {
if err := CanCreateTxnRecord(ctx, cArgs.EvalCtx, reply.Txn); err != nil {
return result.Result{}, err
}
}
} else {
// We're using existingTxn on the reply, although it can be stale
// compared to the Transaction in the request (e.g. the Sequence,
// and various timestamps). We must be careful to update it with the
// supplied ba.Txn if we return it with an error which might be
// retried, as for example to avoid client-side serializable restart.
reply.Txn = &existingTxn
// Verify that we can either commit it or abort it (according
// to args.Commit), and also that the Timestamp and Epoch have
// not suffered regression.
switch reply.Txn.Status {
case roachpb.COMMITTED:
// This can happen if the coordinator had left the transaction in the
// implicitly committed state, and is now coming to clean it up. Someone
// else must have performed the STAGING->COMMITTED transition in the
// meantime. The TransactionStatusError is going to be handled by the
// txnCommitter interceptor.
log.VEventf(ctx, 2, "transaction found to be already committed")
return result.Result{}, roachpb.NewTransactionStatusError(
roachpb.TransactionStatusError_REASON_TXN_COMMITTED,
"already committed")
case roachpb.ABORTED:
if !args.Commit {
// The transaction has already been aborted by other.
// Do not return TransactionAbortedError since the client anyway
// wanted to abort the transaction.
desc := cArgs.EvalCtx.Desc()
resolvedLocks, externalLocks, err := resolveLocalLocks(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx)
if err != nil {
return result.Result{}, err
}
if err := updateFinalizedTxn(
ctx, readWriter, ms, key, args, reply.Txn, recordAlreadyExisted, externalLocks,
); err != nil {
return result.Result{}, err
}
// Use alwaysReturn==true because the transaction is definitely
// aborted, no matter what happens to this command.
res := result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison)
res.Local.ResolvedLocks = resolvedLocks
return res, nil
}
// If the transaction was previously aborted by a concurrent writer's
// push, any intents written are still open. It's only now that we know
// them, so we return them all for asynchronous resolution (we're
// currently not able to write on error, but see #1989).
//
// Similarly to above, use alwaysReturn==true. The caller isn't trying
// to abort, but the transaction is definitely aborted and its locks
// can go.
reply.Txn.LockSpans = args.LockSpans
return result.FromEndTxn(reply.Txn, true /* alwaysReturn */, args.Poison),
roachpb.NewTransactionAbortedError(roachpb.ABORT_REASON_ABORTED_RECORD_FOUND)
case roachpb.PENDING:
if h.Txn.Epoch < reply.Txn.Epoch {
return result.Result{}, errors.AssertionFailedf(
"programming error: epoch regression: %d", h.Txn.Epoch)
}
case roachpb.STAGING:
if h.Txn.Epoch < reply.Txn.Epoch {
return result.Result{}, errors.AssertionFailedf(
"programming error: epoch regression: %d", h.Txn.Epoch)
}
if h.Txn.Epoch > reply.Txn.Epoch {
// If the EndTxn carries a newer epoch than a STAGING txn record, we do
// not consider the transaction to be performing a parallel commit and
// potentially already implicitly committed because we know that the
// transaction restarted since entering the STAGING state.
reply.Txn.Status = roachpb.PENDING
}
default:
return result.Result{}, errors.AssertionFailedf("bad txn status: %s", reply.Txn)
}
// Update the existing txn with the supplied txn.
reply.Txn.Update(h.Txn)
}
// Attempt to commit or abort the transaction per the args.Commit parameter.
if args.Commit {
if retry, reason, extraMsg := IsEndTxnTriggeringRetryError(reply.Txn, args); retry {
return result.Result{}, roachpb.NewTransactionRetryError(reason, extraMsg)
}
// If the transaction needs to be staged as part of an implicit commit
// before being explicitly committed, write the staged transaction
// record and return without running commit triggers or resolving local
// locks.
if args.IsParallelCommit() {
// It's not clear how to combine transaction recovery with commit
// triggers, so for now we don't allow them to mix. This shouldn't
// cause any issues and the txn coordinator knows not to mix them.
if ct := args.InternalCommitTrigger; ct != nil {
err := errors.Errorf("cannot stage transaction with a commit trigger: %+v", ct)
return result.Result{}, err
}
reply.Txn.Status = roachpb.STAGING
reply.StagingTimestamp = reply.Txn.WriteTimestamp
if err := updateStagingTxn(ctx, readWriter, ms, key, args, reply.Txn); err != nil {
return result.Result{}, err
}
return result.Result{}, nil
}
// Else, the transaction can be explicitly committed.
reply.Txn.Status = roachpb.COMMITTED
} else {
// If the transaction is STAGING, we can only move it to ABORTED if it is
// *not* already implicitly committed. On the commit path, the transaction
// coordinator is deliberate to only ever issue an EndTxn(commit) once the
// transaction has reached an implicit commit state. However, on the
// rollback path, the transaction coordinator does not make the opposite
// guarantee that it will never issue an EndTxn(abort) once the transaction
// has reached (or if it still could reach) an implicit commit state.
//
// As a result, on the rollback path, we don't trust the transaction's
// coordinator to be an authoritative source of truth about whether the
// transaction is implicitly committed. In other words, we don't consider
// this EndTxn(abort) to be a claim that the transaction is not implicitly
// committed. The transaction's coordinator may have just given up on the
// transaction before it heard the outcome of a commit attempt. So in this
// case, we return an IndeterminateCommitError to trigger the transaction
// recovery protocol and transition the transaction record to a finalized
// state (COMMITTED or ABORTED).
//
// Interestingly, because intents are not currently resolved until after an
// implicitly committed transaction has been moved to an explicit commit
// state (i.e. its record has moved from STAGING to COMMITTED), no other
// transaction could see the effect of an implicitly committed transaction
// that was erroneously rolled back. This means that such a mistake does not
// actually compromise atomicity. Regardless, such a transition is confusing
// and can cause errors in transaction recovery code. We would also like to
// begin resolving intents earlier, while a transaction is still implicitly
// committed. Doing so is only possible if we can guarantee that under no
// circumstances can an implicitly committed transaction be rolled back.
if reply.Txn.Status == roachpb.STAGING {
err := roachpb.NewIndeterminateCommitError(*reply.Txn)
log.VEventf(ctx, 1, "%v", err)
return result.Result{}, err
}
reply.Txn.Status = roachpb.ABORTED
}
// Resolve locks on the local range synchronously so that their resolution
// ends up in the same Raft entry. There should always be at least one because
// we position the transaction record next to the first write of a transaction.
// This avoids the need for the intentResolver to have to return to this range
// to resolve locks for this transaction in the future.
desc := cArgs.EvalCtx.Desc()
resolvedLocks, externalLocks, err := resolveLocalLocks(ctx, desc, readWriter, ms, args, reply.Txn, cArgs.EvalCtx)
if err != nil {
return result.Result{}, err
}
if err := updateFinalizedTxn(
ctx, readWriter, ms, key, args, reply.Txn, recordAlreadyExisted, externalLocks,
); err != nil {
return result.Result{}, err
}
// Note: there's no need to clear the AbortSpan state if we've successfully
// finalized a transaction, as there's no way in which an abort cache entry
// could have been written (the txn would already have been in
// state=ABORTED).
//
// Summary of transaction replay protection after EndTxn: When a
// transactional write gets replayed over its own resolved intents, the
// write will succeed but only as an intent with a newer timestamp (with a
// WriteTooOldError). However, the replayed intent cannot be resolved by a
// subsequent replay of this EndTxn call because the txn timestamp will be
// too old. Replays of requests which attempt to create a new txn record
// (HeartbeatTxn or EndTxn) never succeed because EndTxn inserts in the
// timestamp cache in Replica's updateTimestampCache method, forcing
// the call to CanCreateTxnRecord to return false, resulting in a
// transaction retry error. If the replay didn't attempt to create a txn
// record, any push will immediately succeed as a missing txn record on push
// where CanCreateTxnRecord returns false succeeds. In both cases, the txn
// will be GC'd on the slow path.
//
// We specify alwaysReturn==false because if the commit fails below Raft, we
// don't want the locks to be up for resolution. That should happen only if
// the commit actually happens; otherwise, we risk losing writes.
txnResult := result.FromEndTxn(reply.Txn, false /* alwaysReturn */, args.Poison)
txnResult.Local.UpdatedTxns = []*roachpb.Transaction{reply.Txn}
txnResult.Local.ResolvedLocks = resolvedLocks
// Run the rest of the commit triggers if successfully committed.
if reply.Txn.Status == roachpb.COMMITTED {
triggerResult, err := RunCommitTrigger(
ctx, cArgs.EvalCtx, readWriter.(storage.Batch), ms, args, reply.Txn,
)
if err != nil {
return result.Result{}, err
}
if err := txnResult.MergeAndDestroy(triggerResult); err != nil {
return result.Result{}, err
}
}
return txnResult, nil
}
// IsEndTxnExceedingDeadline returns true if the transaction's provisional
// commit timestamp exceeded its deadline. If so, the transaction should not be
// allowed to commit.
func IsEndTxnExceedingDeadline(commitTS hlc.Timestamp, deadline hlc.Timestamp) bool {
return !deadline.IsEmpty() && deadline.LessEq(commitTS)
}
// IsEndTxnTriggeringRetryError returns true if the EndTxnRequest cannot be
// committed and needs to return a TransactionRetryError. It also returns the
// reason and possibly an extra message to be used for the error.
func IsEndTxnTriggeringRetryError(
txn *roachpb.Transaction, args *roachpb.EndTxnRequest,
) (retry bool, reason roachpb.TransactionRetryReason, extraMsg string) {
// If we saw any WriteTooOldErrors, we must restart to avoid lost
// update anomalies.
if txn.WriteTooOld {
retry, reason = true, roachpb.RETRY_WRITE_TOO_OLD
} else {
readTimestamp := txn.ReadTimestamp
isTxnPushed := txn.WriteTimestamp != readTimestamp
// Return a transaction retry error if the commit timestamp isn't equal to
// the txn timestamp.
if isTxnPushed {
retry, reason = true, roachpb.RETRY_SERIALIZABLE
}
}
// A transaction must obey its deadline, if set.
if !retry && IsEndTxnExceedingDeadline(txn.WriteTimestamp, args.Deadline) {
exceededBy := txn.WriteTimestamp.GoTime().Sub(args.Deadline.GoTime())
extraMsg = fmt.Sprintf(
"txn timestamp pushed too much; deadline exceeded by %s (%s > %s)",
exceededBy, txn.WriteTimestamp, args.Deadline)
retry, reason = true, roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED
}
return retry, reason, extraMsg
}
const lockResolutionBatchSize = 500
// resolveLocalLocks synchronously resolves any locks that are local to this
// range in the same batch and returns those lock spans. The remainder are
// collected and returned so that they can be handed off to asynchronous
// processing. Note that there is a maximum lock resolution allowance of
// lockResolutionBatchSize meant to avoid creating a batch which is too large
// for Raft. Any local locks which exceed the allowance are treated as
// external and are resolved asynchronously with the external locks.
func resolveLocalLocks(
ctx context.Context,
desc *roachpb.RangeDescriptor,
readWriter storage.ReadWriter,
ms *enginepb.MVCCStats,
args *roachpb.EndTxnRequest,
txn *roachpb.Transaction,
evalCtx EvalContext,
) (resolvedLocks []roachpb.LockUpdate, externalLocks []roachpb.Span, _ error) {
if mergeTrigger := args.InternalCommitTrigger.GetMergeTrigger(); mergeTrigger != nil {
// If this is a merge, then use the post-merge descriptor to determine
// which locks are local (note that for a split, we want to use the
// pre-split one instead because it's larger).
desc = &mergeTrigger.LeftDesc
}
var resolveAllowance int64 = lockResolutionBatchSize
if args.InternalCommitTrigger != nil {
// If this is a system transaction (such as a split or merge), don't enforce the resolve allowance.
// These transactions rely on having their locks resolved synchronously.
resolveAllowance = math.MaxInt64
}
for _, span := range args.LockSpans {
if err := func() error {
if resolveAllowance == 0 {
externalLocks = append(externalLocks, span)
return nil
}
update := roachpb.MakeLockUpdate(txn, span)
if len(span.EndKey) == 0 {
// For single-key lock updates, do a KeyAddress-aware check of
// whether it's contained in our Range.
if !kvserverbase.ContainsKey(desc, span.Key) {
externalLocks = append(externalLocks, span)
return nil
}
// It may be tempting to reuse an iterator here, but this call
// can create the iterator with Prefix:true which is much faster
// than seeking -- especially for intents that are missing, e.g.
// due to async intent resolution. See:
// https://github.com/cockroachdb/cockroach/issues/64092
//
// Note that the underlying pebbleIterator will still be reused
// since readWriter is a pebbleBatch in the typical case.
ok, err := storage.MVCCResolveWriteIntent(ctx, readWriter, ms, update)
if err != nil {
return err
}
if ok {
resolveAllowance--
}
resolvedLocks = append(resolvedLocks, update)
return nil
}
// For update ranges, cut into parts inside and outside our key
// range. Resolve locally inside, delegate the rest. In particular,
// an update range for range-local data is correctly considered local.
inSpan, outSpans := kvserverbase.IntersectSpan(span, desc)
externalLocks = append(externalLocks, outSpans...)
if inSpan != nil {
update.Span = *inSpan
num, resumeSpan, err := storage.MVCCResolveWriteIntentRange(
ctx, readWriter, ms, update, resolveAllowance)
if err != nil {
return err
}
if evalCtx.EvalKnobs().NumKeysEvaluatedForRangeIntentResolution != nil {
atomic.AddInt64(evalCtx.EvalKnobs().NumKeysEvaluatedForRangeIntentResolution, num)
}
resolveAllowance -= num
if resumeSpan != nil {
if resolveAllowance != 0 {
log.Fatalf(ctx, "expected resolve allowance to be exactly 0 resolving %s; got %d", update.Span, resolveAllowance)
}
update.EndKey = resumeSpan.Key
externalLocks = append(externalLocks, *resumeSpan)
}
resolvedLocks = append(resolvedLocks, update)
return nil
}
return nil
}(); err != nil {
return nil, nil, errors.Wrapf(err, "resolving lock at %s on end transaction [%s]", span, txn.Status)
}
}
removedAny := resolveAllowance != lockResolutionBatchSize
if WriteAbortSpanOnResolve(txn.Status, args.Poison, removedAny) {
if err := UpdateAbortSpan(ctx, evalCtx, readWriter, ms, txn.TxnMeta, args.Poison); err != nil {
return nil, nil, err
}
}
return resolvedLocks, externalLocks, nil
}
// updateStagingTxn persists the STAGING transaction record with updated status
// (and possibly timestamp). It persists the record with the EndTxn request's
// declared in-flight writes along with all of the transaction's (local and
// remote) locks.
func updateStagingTxn(
ctx context.Context,
readWriter storage.ReadWriter,
ms *enginepb.MVCCStats,
key []byte,
args *roachpb.EndTxnRequest,
txn *roachpb.Transaction,
) error {
txn.LockSpans = args.LockSpans
txn.InFlightWrites = args.InFlightWrites
txnRecord := txn.AsRecord()
return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord)
}
// updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with
// updated status (and possibly timestamp). If we've already resolved all locks
// locally, we actually delete the record right away - no use in keeping it
// around.
func updateFinalizedTxn(
ctx context.Context,
readWriter storage.ReadWriter,
ms *enginepb.MVCCStats,
key []byte,
args *roachpb.EndTxnRequest,
txn *roachpb.Transaction,
recordAlreadyExisted bool,
externalLocks []roachpb.Span,
) error {
if txnAutoGC && len(externalLocks) == 0 {
if log.V(2) {
log.Infof(ctx, "auto-gc'ed %s (%d locks)", txn.Short(), len(args.LockSpans))
}
if !recordAlreadyExisted {
// Nothing to delete, so there's no use writing a deletion tombstone. This
// can help avoid sending a proposal through Raft, if nothing else in the
// BatchRequest writes.
return nil
}
_, err := storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil)
return err
}
txn.LockSpans = externalLocks
txn.InFlightWrites = nil
txnRecord := txn.AsRecord()
return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord)
}
// RunCommitTrigger runs the commit trigger from an end transaction request.
func RunCommitTrigger(
ctx context.Context,
rec EvalContext,
batch storage.Batch,
ms *enginepb.MVCCStats,
args *roachpb.EndTxnRequest,
txn *roachpb.Transaction,
) (result.Result, error) {
ct := args.InternalCommitTrigger
if ct == nil {
return result.Result{}, nil
}
// The transaction is committing with a commit trigger. This means that it has
// side-effects beyond those of the intents that it has written.
// The transaction should not have a commit timestamp in the future of present
// time. Such cases should be caught in maybeCommitWaitBeforeCommitTrigger
// before getting here, which should sleep for long enough to ensure that the
// local clock leads the commit timestamp. An error here may indicate that the
// transaction's commit timestamp was bumped after it acquired latches.
if rec.Clock().Now().Less(txn.WriteTimestamp) {
return result.Result{}, errors.AssertionFailedf("txn %s with %s commit trigger needs "+
"commit wait. Was its timestamp bumped after acquiring latches?", txn, ct.Kind())
}
// Stage the commit trigger's side-effects so that they will go into effect on
// each Replica when the corresponding Raft log entry is applied. Only one
// commit trigger can be set.
if ct.GetSplitTrigger() != nil {
newMS, res, err := splitTrigger(
ctx, rec, batch, *ms, ct.SplitTrigger, txn.WriteTimestamp,
)
if err != nil {
return result.Result{}, roachpb.NewReplicaCorruptionError(err)
}
*ms = newMS
return res, nil
}
if mt := ct.GetMergeTrigger(); mt != nil {
res, err := mergeTrigger(ctx, rec, batch, ms, mt, txn.WriteTimestamp)
if err != nil {
return result.Result{}, roachpb.NewReplicaCorruptionError(err)
}
return res, nil
}
if crt := ct.GetChangeReplicasTrigger(); crt != nil {
// TODO(tbg): once we support atomic replication changes, check that
// crt.Added() and crt.Removed() don't intersect (including mentioning
// the same replica more than once individually) because it would be
// silly (though possible) to have to attach semantics to that.
return changeReplicasTrigger(ctx, rec, batch, crt), nil
}
if ct.GetModifiedSpanTrigger() != nil {
var pd result.Result
if nlSpan := ct.ModifiedSpanTrigger.NodeLivenessSpan; nlSpan != nil {
if err := pd.MergeAndDestroy(
result.Result{
Local: result.LocalResult{
MaybeGossipNodeLiveness: nlSpan,
},
},
); err != nil {
return result.Result{}, err
}
}
return pd, nil
}
if sbt := ct.GetStickyBitTrigger(); sbt != nil {
newDesc := *rec.Desc()
newDesc.StickyBit = sbt.StickyBit
var res result.Result
res.Replicated.State = &kvserverpb.ReplicaState{
Desc: &newDesc,
}
return res, nil
}
log.Fatalf(ctx, "unknown commit trigger: %+v", ct)
return result.Result{}, nil
}
// splitTrigger is called on a successful commit of a transaction containing an
// AdminSplit operation. It copies the AbortSpan for the new range and
// recomputes stats for both the existing, left hand side (LHS) range and the
// right hand side (RHS) range. For performance it only computes the stats for
// one side of the range and infers the stats for the other side by subtracting
// from the original stats. The choice of which side to scan is controlled by a
// heuristic. This choice defaults to scanning the LHS stats and inferring the
// RHS because the split key computation performed by the splitQueue ensures
// that we do not create large LHS ranges. However, if the RHS's global keyspace
// is entirely empty, it is scanned first instead. An example where we expect
// this heuristic to choose the RHS is bulk ingestion, which often splits off
// empty ranges and benefits from scanning the empty RHS when computing stats.
// Regardless of the choice of which side to scan first, the optimization to
// infer the other side's stats is only possible if the stats are fully accurate
// (ContainsEstimates = 0). If they contain estimates, stats for both the LHS
// and RHS are computed.
//
// Splits are complicated. A split is initiated when a replica receives an
// AdminSplit request. Note that this request (and other "admin" requests)
// differs from normal requests in that it doesn't go through Raft but instead
// allows the lease holder Replica to act as the orchestrator for the
// distributed transaction that performs the split. As such, this request is
// only executed on the lease holder replica and the request is redirected to
// the lease holder if the recipient is a follower.
//
// Splits do not require the lease for correctness (which is good, because we
// only check that the lease is held at the beginning of the operation, and
// have no way to ensure that it is continually held until the end). Followers
// could perform splits too, and the only downside would be that if two splits
// were attempted concurrently (or a split and a ChangeReplicas), one would
// fail. The lease is used to designate one replica for this role and avoid
// wasting time on splits that may fail.
//
// The processing of splits is divided into two phases. The first phase occurs
// in Replica.AdminSplit. In that phase, the split-point is computed, and a
// transaction is started which updates both the LHS and RHS range descriptors
// and the meta range addressing information. (If we're splitting a meta2 range
// we'll be updating the meta1 addressing, otherwise we'll be updating the
// meta2 addressing). That transaction includes a special SplitTrigger flag on
// the EndTxn request. Like all transactions, the requests within the
// transaction are replicated via Raft, including the EndTxn request.
//
// The second phase of split processing occurs when each replica for the range
// encounters the SplitTrigger. Processing of the SplitTrigger happens below,
// in Replica.splitTrigger. The processing of the SplitTrigger occurs in two
// stages. The first stage operates within the context of an engine.Batch and
// updates all of the on-disk state for the old and new ranges atomically. The
// second stage is invoked when the batch commits and updates the in-memory
// state, creating the new replica in memory and populating its timestamp cache
// and registering it with the store.
//
// There is lots of subtlety here. The easy scenario is that all of the
// replicas process the SplitTrigger before processing any Raft message for RHS
// (right hand side) of the newly split range. Something like:
//
// Node A Node B Node C
// ----------------------------------------------------
// range 1 | | |
// | | |
// SplitTrigger | |
// | SplitTrigger |
// | | SplitTrigger
// | | |
// ----------------------------------------------------
// split finished on A, B and C | |
// | | |
// range 2 | | |
// | ---- MsgVote --> | |
// | ---------------------- MsgVote ---> |
//
// But that ideal ordering is not guaranteed. The split is "finished" when two
// of the replicas have appended the end-txn request containing the
// SplitTrigger to their Raft log. The following scenario is possible:
//
// Node A Node B Node C
// ----------------------------------------------------
// range 1 | | |
// | | |
// SplitTrigger | |
// | SplitTrigger |
// | | |
// ----------------------------------------------------
// split finished on A and B | |
// | | |
// range 2 | | |
// | ---- MsgVote --> | |
// | --------------------- MsgVote ---> ???
// | | |
// | | SplitTrigger
//
// In this scenario, C will create range 2 upon reception of the MsgVote from
// A, though locally that span of keys is still part of range 1. This is
// possible because at the Raft level ranges are identified by integer IDs and
// it isn't until C receives a snapshot of range 2 from the leader that it
// discovers the span of keys it covers. In order to prevent C from fully
// initializing range 2 in this instance, we prohibit applying a snapshot to a
// range if the snapshot overlaps another range. See Store.canApplySnapshotLocked.
//
// But while a snapshot may not have been applied at C, an uninitialized
// Replica was created. An uninitialized Replica is one which belongs to a Raft
// group but for which the range descriptor has not been received. This Replica
// will have participated in the Raft elections. When we're creating the new
// Replica below we take control of this uninitialized Replica and stop it from
// responding to Raft messages by marking it "destroyed". Note that we use the
// Replica.mu.destroyed field for this, but we don't do everything that
// Replica.Destroy does (so we should probably rename that field in light of
// its new uses). In particular we don't touch any data on disk or leave a
// tombstone. This is especially important because leaving a tombstone would
// prevent the legitimate recreation of this replica.
//
// There is subtle synchronization here that is currently controlled by the
// Store.processRaft goroutine. In particular, the serial execution of
// Replica.handleRaftReady by Store.processRaft ensures that an uninitialized
// RHS won't be concurrently executing in Replica.handleRaftReady because we're
// currently running on that goroutine (i.e. Replica.splitTrigger is called on
// the processRaft goroutine).
//
// TODO(peter): The above synchronization needs to be fixed. Using a single
// goroutine for executing Replica.handleRaftReady is undesirable from a
// performance perspective. Likely we will have to add a mutex to Replica to
// protect handleRaftReady and to grab that mutex below when marking the
// uninitialized Replica as "destroyed". Hopefully we'll also be able to remove
// Store.processRaftMu.
//
// Note that in this more complex scenario, A (which performed the SplitTrigger
// first) will create the associated Raft group for range 2 and start
// campaigning immediately. It is possible for B to receive MsgVote requests
// before it has applied the SplitTrigger as well. Both B and C will vote for A
// (and preserve the records of that vote in their HardState). It is critically
// important for Raft correctness that we do not lose the records of these
// votes. After electing A the Raft leader for range 2, A will then attempt to
// send a snapshot to B and C and we'll fall into the situation above where a
// snapshot is received for a range before it has finished splitting from its
// sibling and is thus rejected. An interesting subtlety here: A will send a
// snapshot to B and C because when range 2 is initialized we were careful set
// synthesize its HardState to set its Raft log index to 10. If we had instead
// used log index 0, Raft would have believed the group to be empty, but the
// RHS has something. Using a non-zero initial log index causes Raft to believe
// that there is a discarded prefix to the log and will thus send a snapshot to
// followers.
//
// A final point of clarification: when we split a range we're splitting the
// data the range contains. But we're not forking or splitting the associated
// Raft group. Instead, we're creating a new Raft group to control the RHS of
// the split. That Raft group is starting from an empty Raft log (positioned at
// log entry 10) and a snapshot of the RHS of the split range.
//
// After the split trigger returns, the on-disk state of the right-hand side
// will be suitable for instantiating the right hand side Replica, and
// a suitable trigger is returned, along with the updated stats which represent
// the LHS delta caused by the split (i.e. all writes in the current batch
// which went to the left-hand side, minus the kv pairs which moved to the
// RHS).
//
// These stats are suitable for returning up the callstack like those for
// regular commands; the corresponding delta for the RHS is part of the
// returned trigger and is handled by the Store.
func splitTrigger(
ctx context.Context,
rec EvalContext,
batch storage.Batch,
bothDeltaMS enginepb.MVCCStats,
split *roachpb.SplitTrigger,
ts hlc.Timestamp,
) (enginepb.MVCCStats, result.Result, error) {
desc := rec.Desc()
if !bytes.Equal(desc.StartKey, split.LeftDesc.StartKey) ||
!bytes.Equal(desc.EndKey, split.RightDesc.EndKey) {
return enginepb.MVCCStats{}, result.Result{}, errors.Errorf("range does not match splits: (%s-%s) + (%s-%s) != %s",
split.LeftDesc.StartKey, split.LeftDesc.EndKey,
split.RightDesc.StartKey, split.RightDesc.EndKey, desc)
}
// Determine which side to scan first when computing the post-split stats. We
// scan the left-hand side first unless the right side's global keyspace is
// entirely empty. In cases where the range's stats do not already contain
// estimates, only one side needs to be scanned.
// TODO(nvanbenschoten): this is a simple heuristic. If we had a cheap way to
// determine the relative sizes of the LHS and RHS, we could be more
// sophisticated here and always choose to scan the cheaper side.
emptyRHS, err := isGlobalKeyspaceEmpty(batch, &split.RightDesc)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrapf(err,
"unable to determine whether right hand side of split is empty")
}
rangeKeyDeltaMS, err := computeSplitRangeKeyStatsDelta(batch, split.LeftDesc, split.RightDesc)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err,
"unable to compute range key stats delta for RHS")
}
h := splitStatsHelperInput{
AbsPreSplitBothEstimated: rec.GetMVCCStats(),
DeltaBatchEstimated: bothDeltaMS,
DeltaRangeKey: rangeKeyDeltaMS,
AbsPostSplitLeftFn: makeScanStatsFn(ctx, batch, ts, &split.LeftDesc, "left hand side"),
AbsPostSplitRightFn: makeScanStatsFn(ctx, batch, ts, &split.RightDesc, "right hand side"),
ScanRightFirst: splitScansRightForStatsFirst || emptyRHS,
}
return splitTriggerHelper(ctx, rec, batch, h, split, ts)
}
// splitScansRightForStatsFirst controls whether the left hand side or the right
// hand side of the split is scanned first on the leaseholder when evaluating
// the split trigger. In practice, the splitQueue wants to scan the left hand
// side because the split key computation ensures that we do not create large
// LHS ranges. However, to improve test coverage, we use a metamorphic value.
var splitScansRightForStatsFirst = util.ConstantWithMetamorphicTestBool(
"split-scans-right-for-stats-first", false)
// isGlobalKeyspaceEmpty returns whether the global keyspace of the provided
// range is entirely empty. The function returns false if the global keyspace
// contains at least one key.
func isGlobalKeyspaceEmpty(reader storage.Reader, d *roachpb.RangeDescriptor) (bool, error) {
span := d.KeySpan().AsRawSpanWithNoLocals()
iter := reader.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{UpperBound: span.EndKey})
defer iter.Close()
iter.SeekGE(storage.MakeMVCCMetadataKey(span.Key))
ok, err := iter.Valid()
if err != nil {
return false, err
}
return !ok /* empty */, nil
}
// makeScanStatsFn constructs a splitStatsScanFn for the provided post-split
// range descriptor which computes the range's statistics.
func makeScanStatsFn(
ctx context.Context,
reader storage.Reader,
ts hlc.Timestamp,
sideDesc *roachpb.RangeDescriptor,
sideName string,
) splitStatsScanFn {
return func() (enginepb.MVCCStats, error) {
sideMS, err := rditer.ComputeStatsForRange(sideDesc, reader, ts.WallTime)
if err != nil {
return enginepb.MVCCStats{}, errors.Wrapf(err,
"unable to compute stats for %s range after split", sideName)
}
log.Eventf(ctx, "computed stats for %s range", sideName)
return sideMS, nil
}
}
// splitTriggerHelper continues the work begun by splitTrigger, but has a
// reduced scope that has all stats-related concerns bundled into a
// splitStatsHelper.
func splitTriggerHelper(
ctx context.Context,
rec EvalContext,
batch storage.Batch,
statsInput splitStatsHelperInput,
split *roachpb.SplitTrigger,
ts hlc.Timestamp,
) (enginepb.MVCCStats, result.Result, error) {
// TODO(d4l3k): we should check which side of the split is smaller
// and compute stats for it instead of having a constraint that the
// left hand side is smaller.
// NB: the replicated post-split left hand keyspace is frozen at this point.
// Only the RHS can be mutated (and we do so to seed its state).
// Copy the last replica GC timestamp. This value is unreplicated,
// which is why the MVCC stats are set to nil on calls to
// MVCCPutProto.
replicaGCTS, err := rec.GetLastReplicaGCTimestamp(ctx)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp")
}
if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &replicaGCTS); err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp")
}
// Compute the absolute stats for the (post-split) ranges. No more
// modifications to the left hand side are allowed after this line and any