@@ -13,6 +13,7 @@ import (
1313 "reflect"
1414 "sort"
1515 "strings"
16+ "sync"
1617 "sync/atomic"
1718 "testing"
1819 "time"
@@ -25,7 +26,9 @@ import (
2526 "github.com/cockroachdb/cockroach/pkg/kv/kvpb"
2627 "github.com/cockroachdb/cockroach/pkg/kv/kvserver"
2728 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
29+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
2830 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation"
31+ "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
2932 "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
3033 "github.com/cockroachdb/cockroach/pkg/roachpb"
3134 "github.com/cockroachdb/cockroach/pkg/rpc"
@@ -4643,3 +4646,173 @@ func TestProxyTracing(t *testing.T) {
46434646 t .Logf ("found trace event; msg=%s, tag=%s, loc=%s" , msg , tag , loc )
46444647 })
46454648}
4649+
4650+ // TestUnexpectedCommitOnTxnRecovery constructs a scenario where transaction
4651+ // recovery could incorrectly determine that a transaction is committed. The
4652+ // scenario is as follows:
4653+ //
4654+ // Txn1:
4655+ // - Writes to keyA.
4656+ // - Acquires an unreplicated exclusive lock on keyB.
4657+ // - Acquires a replicated shared lock on keyB. This lock is pipelined, and
4658+ // replication for it fails.
4659+ // - Attempts to commit, but fails because of the lost replicated Shared lock.
4660+ //
4661+ // Lease is then transferred to n3. This causes the unreplicated exclusive lock
4662+ // on keyB to be replicated.
4663+ //
4664+ // Txn2:
4665+ // - Attempts to read keyA, which kicks off transaction recovery for Txn1.
4666+ // - Txn2 (incorrectly) concludes that Txn1 is committed at epoch=1 because it
4667+ // finds a (stronger than Shared) replicated lock on keyB.
4668+ //
4669+ // Txn1:
4670+ // - Back here, we do a stateful retry. We should learn that someone (Txn2)
4671+ // aborted us when we go and try to commit. At the time of writing, we
4672+ // incorrectly learn that we've been (unexpectedly) committed.
4673+ func TestUnexpectedCommitOnTxnRecovery (t * testing.T ) {
4674+ defer leaktest .AfterTest (t )()
4675+ defer log .Scope (t ).Close (t )
4676+
4677+ keyA := roachpb .Key ("a" )
4678+ keyB := roachpb .Key ("b" )
4679+
4680+ var (
4681+ targetTxnIDString atomic.Value
4682+ cmdID atomic.Value
4683+ )
4684+ cmdID .Store (kvserverbase .CmdIDKey ("" ))
4685+ targetTxnIDString .Store ("" )
4686+ ctx := context .Background ()
4687+ st := cluster .MakeTestingClusterSettings ()
4688+ // This test relies on unreplicated locks to be replicated on lease transfers.
4689+ concurrency .UnreplicatedLockReliabilityLeaseTransfer .Override (ctx , & st .SV , true )
4690+ tc := testcluster .StartTestCluster (t , 3 , base.TestClusterArgs {
4691+ ServerArgs : base.TestServerArgs {
4692+ Settings : st ,
4693+ Knobs : base.TestingKnobs {
4694+ Store : & kvserver.StoreTestingKnobs {
4695+ TestingProposalFilter : func (fArgs kvserverbase.ProposalFilterArgs ) * kvpb.Error {
4696+ if fArgs .Req .Header .Txn == nil ||
4697+ fArgs .Req .Header .Txn .ID .String () != targetTxnIDString .Load ().(string ) {
4698+ return nil // not our txn
4699+ }
4700+ if ! fArgs .Req .IsSingleRequest () {
4701+ // Not the request we care about.
4702+ return nil
4703+ }
4704+ getReq , ok := fArgs .Req .Requests [0 ].GetInner ().(* kvpb.GetRequest )
4705+ // Only fail replication on the first retry.
4706+ epoch := fArgs .Req .Header .Txn .Epoch
4707+ if ok && getReq .KeyLockingDurability == lock .Replicated && epoch == 0 {
4708+ t .Logf ("will fail application for txn %s@epoch=%d; req: %+v; raft cmdID: %s" ,
4709+ fArgs .Req .Header .Txn .ID .String (), epoch , getReq , fArgs .CmdID )
4710+ cmdID .Store (fArgs .CmdID )
4711+ }
4712+ return nil
4713+ },
4714+ TestingApplyCalledTwiceFilter : func (fArgs kvserverbase.ApplyFilterArgs ) (int , * kvpb.Error ) {
4715+ if fArgs .CmdID == cmdID .Load ().(kvserverbase.CmdIDKey ) {
4716+ t .Logf ("failing application for raft cmdID: %s" , cmdID )
4717+
4718+ return 0 , kvpb .NewErrorf ("test injected error" )
4719+ }
4720+ return 0 , nil
4721+ },
4722+ },
4723+ },
4724+ },
4725+ })
4726+ defer tc .Stopper ().Stop (ctx )
4727+
4728+ transferLease := func (idx int ) {
4729+ desc := tc .LookupRangeOrFatal (t , keyB )
4730+ tc .TransferRangeLeaseOrFatal (t , desc , tc .Target (idx ))
4731+ }
4732+ // Make a db with transaction heartbeating disabled. This ensures that we
4733+ // don't mark Txn1 as PENDING after its first failed parallel commit attempt,
4734+ // which would otherwise prevent Txn2 from recovering Txn1.
4735+ s := tc .Server (0 )
4736+ ambient := s .AmbientCtx ()
4737+ tsf := kvcoord .NewTxnCoordSenderFactory (
4738+ kvcoord.TxnCoordSenderFactoryConfig {
4739+ AmbientCtx : ambient ,
4740+ HeartbeatInterval : - 1 , // disabled
4741+ Settings : s .ClusterSettings (),
4742+ Clock : s .Clock (),
4743+ Stopper : s .Stopper (),
4744+ },
4745+ s .DistSenderI ().(* kvcoord.DistSender ),
4746+ )
4747+ db := kv .NewDB (ambient , tsf , s .Clock (), s .Stopper ())
4748+
4749+ startTxn2 := make (chan struct {})
4750+ blockCh := make (chan struct {})
4751+ var wg sync.WaitGroup
4752+ wg .Add (1 )
4753+
4754+ // Write to keyB so that we can later get a lock on it.
4755+ txn := db .NewTxn (ctx , "txn" )
4756+ err := txn .Put (ctx , keyB , "valueB" )
4757+ require .NoError (t , err )
4758+ require .NoError (t , txn .Commit (ctx ))
4759+
4760+ go func () {
4761+ defer wg .Done ()
4762+
4763+ err := db .Txn (ctx , func (ctx context.Context , txn * kv.Txn ) error {
4764+ if txnID := targetTxnIDString .Load (); txnID == "" {
4765+ // Store the txnID for the testing knobs.
4766+ targetTxnIDString .Store (txn .ID ().String ())
4767+ t .Logf ("txn1 ID is: %s" , txn .ID ())
4768+ } else if txnID != txn .ID () {
4769+ // Since txn recovery aborted us, we get retried again but with an
4770+ // entirely new txnID. This time we just return. Writing nothing.
4771+ return nil
4772+ }
4773+
4774+ switch txn .Epoch () {
4775+ case 0 :
4776+ err := txn .Put (ctx , keyA , "value" )
4777+ require .NoError (t , err )
4778+ res , err := txn .GetForUpdate (ctx , keyB , kvpb .BestEffort )
4779+ require .NoError (t , err )
4780+ require .Equal (t , res .ValueBytes (), []byte ("valueB" ))
4781+ res , err = txn .GetForShare (ctx , keyB , kvpb .GuaranteedDurability )
4782+ require .NoError (t , err )
4783+ require .Equal (t , res .ValueBytes (), []byte ("valueB" ))
4784+ err = txn .Commit (ctx )
4785+ require .Error (t , err )
4786+ require .ErrorContains (t , err , "RETRY_ASYNC_WRITE_FAILURE" )
4787+ // Transfer the lease to n3.
4788+ transferLease (2 )
4789+ close (startTxn2 )
4790+ // Block until Txn2 recovers us.
4791+ <- blockCh
4792+ return err
4793+ case 1 :
4794+ // When retrying the write failure we should discover that txn recovery
4795+ // has aborted this transaction.
4796+ err := txn .Put (ctx , keyA , "value" )
4797+ require .Error (t , err )
4798+ require .ErrorContains (t , err , "ABORT_REASON_ABORT_SPAN" )
4799+ return err
4800+ default :
4801+ t .Errorf ("unexpected epoch: %d" , txn .Epoch ())
4802+ }
4803+ return nil
4804+ })
4805+ require .NoError (t , err )
4806+ }()
4807+ <- startTxn2
4808+
4809+ txn2 := db .NewTxn (ctx , "txn2" )
4810+ res , err := txn2 .Get (ctx , keyA )
4811+ require .NoError (t , err )
4812+ // NB: Nothing should exist on keyA, because txn1 didn't commit at epoch 1 (or
4813+ // any epoch, for that matter).
4814+ require .False (t , res .Exists ())
4815+
4816+ close (blockCh )
4817+ wg .Wait ()
4818+ }
0 commit comments