From 111863c186c1c73cc1502e162dcc8433dcbeb79a Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 8 Dec 2021 11:21:47 +0100 Subject: [PATCH] kvcoord: make "txn unexpectedly committed" assertion opt-out We are currently investigating how we are hitting this assertion, but while it fires it is doing more harm than good. See https://github.com/cockroachdb/cockroach/issues/67765. The provided test exercises the case in which the error is returned (i.e. opted out of the assertion). Touches https://github.com/cockroachlabs/support/issues/1315. Release note: None --- .../kvclient/kvcoord/replayed_commit_test.go | 110 ++++++++++++++++++ pkg/kv/kvclient/kvcoord/txn_coord_sender.go | 42 +++++-- 2 files changed, 143 insertions(+), 9 deletions(-) create mode 100644 pkg/kv/kvclient/kvcoord/replayed_commit_test.go diff --git a/pkg/kv/kvclient/kvcoord/replayed_commit_test.go b/pkg/kv/kvclient/kvcoord/replayed_commit_test.go new file mode 100644 index 000000000000..3e8a77dae8ce --- /dev/null +++ b/pkg/kv/kvclient/kvcoord/replayed_commit_test.go @@ -0,0 +1,110 @@ +// Copyright 2021 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 kvcoord_test + +import ( + "context" + "testing" + + circuit "github.com/cockroachdb/circuitbreaker" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type interceptingTransport struct { + kvcoord.Transport + intercept func(context.Context, roachpb.BatchRequest, *roachpb.BatchResponse, error) (*roachpb.BatchResponse, error) +} + +func (f *interceptingTransport) SendNext( + ctx context.Context, ba roachpb.BatchRequest, +) (*roachpb.BatchResponse, error) { + br, err := f.Transport.SendNext(ctx, ba) + return f.intercept(ctx, ba, br, err) +} + +// TestCommitSanityCheckAssertionFiresOnUndetectedAmbiguousCommit sets up a situation +// in which DistSender retries an (unbeknownst to it) successful EndTxn(commit=true) +// RPC. It documents that this triggers an assertion failure in TxnCoordSender. +// +// See: https://github.com/cockroachdb/cockroach/issues/67765 +func TestCommitSanityCheckAssertionFiresOnUndetectedAmbiguousCommit(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + var args base.TestClusterArgs + args.ServerArgs.Knobs.KVClient = &kvcoord.ClientTestingKnobs{TransportFactory: func( + options kvcoord.SendOptions, + dialer *nodedialer.Dialer, + slice kvcoord.ReplicaSlice, + ) (kvcoord.Transport, error) { + tf, err := kvcoord.GRPCTransportFactory(options, dialer, slice) + if err != nil { + return nil, err + } + return &interceptingTransport{ + Transport: tf, + intercept: func(ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, err error) (*roachpb.BatchResponse, error) { + if err != nil || ba.Txn == nil || br.Txn == nil || + ba.Txn.Status != roachpb.PENDING || br.Txn.Status != roachpb.COMMITTED || + !keys.ScratchRangeMin.Equal(br.Txn.Key) { + // Only want to inject error on successful commit for "our" txn. + return br, err + } + err = circuit.ErrBreakerOpen + assert.True(t, grpcutil.RequestDidNotStart(err)) // avoid Fatal on goroutine + return nil, err + }, + }, nil + }, + } + + tc := testcluster.StartTestCluster(t, 1, args) + defer tc.Stopper().Stop(ctx) + + // Txn record GC populates txn tscache which prevents second commit + // attempt from hitting TransactionStatusError(alreadyCommitted). + defer batcheval.TestingSetTxnAutoGC(false)() + { + // Turn the assertion into an error. + prev := kvcoord.DisableCommitSanityCheck + kvcoord.DisableCommitSanityCheck = true + defer func() { + kvcoord.DisableCommitSanityCheck = prev + }() + } + + k := tc.ScratchRange(t) + err := tc.Server(0).DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + _ = txn.DisablePipelining() // keep it simple + if err := txn.Put(ctx, k, "hello"); err != nil { + t.Log(err) + return err + } + err := txn.Commit(ctx) // hits fatal error + t.Log(err) + return err + }) + require.True(t, errors.IsAssertionFailure(err), "%+v", err) +} diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 6f137037df90..27888549a0ee 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -36,6 +37,12 @@ const ( OpTxnCoordSender = "txn coordinator send" ) +// DisableCommitSanityCheck allows opting out of a fatal assertion error that was observed in the wild +// and for which a root cause is not yet available. +// +// See: https://github.com/cockroachdb/cockroach/pull/73512. +var DisableCommitSanityCheck = envutil.EnvOrDefaultBool("COCKROACH_DISABLE_COMMIT_SANITY_CHECK", false) + // txnState represents states relating to whether an EndTxn request needs // to be sent. //go:generate stringer -type=txnState @@ -795,7 +802,7 @@ func (tc *TxnCoordSender) updateStateLocked( // Update our transaction with any information the error has. if errTxn := pErr.GetTxn(); errTxn != nil { if errTxn.Status == roachpb.COMMITTED { - sanityCheckCommittedErr(ctx, pErr, ba) + pErr = sanityCheckCommittedErr(ctx, pErr, ba) } tc.mu.txn.Update(errTxn) } @@ -807,22 +814,39 @@ func (tc *TxnCoordSender) updateStateLocked( // encountering such errors. Marking a transaction as explicitly-committed can // also encounter these errors, but those errors don't make it to the // TxnCoordSender. -func sanityCheckCommittedErr(ctx context.Context, pErr *roachpb.Error, ba roachpb.BatchRequest) { - errTxn := pErr.GetTxn() - if errTxn == nil || errTxn.Status != roachpb.COMMITTED { - // We shouldn't have been called. - return - } +// +// Returns the passed-in error or fatals (depending on DisableCommitSanityCheck env var), +// wrapping the input error in case of an assertion violation. +// +// Requires: pErrWithCommittedTxn is non-nil and GetTxn() is also non-nil. +func sanityCheckCommittedErr( + ctx context.Context, pErrWithCommittedTxn *roachpb.Error, ba roachpb.BatchRequest, +) *roachpb.Error { // The only case in which an error can have a COMMITTED transaction in it is // when the request was a rollback. Rollbacks can race with commits if a // context timeout expires while a commit request is in flight. if ba.IsSingleAbortTxnRequest() { - return + return pErrWithCommittedTxn } // Finding out about our transaction being committed indicates a serious bug. // Requests are not supposed to be sent on transactions after they are // committed. - log.Fatalf(ctx, "transaction unexpectedly committed: %s. ba: %s. txn: %s.", pErr, ba, errTxn) + err := errors.Wrapf(pErrWithCommittedTxn.GoError(), + "transaction unexpectedly committed, ba: %s. txn: %s", + ba, pErrWithCommittedTxn.GetTxn(), + ) + err = errors.WithAssertionFailure( + errors.WithIssueLink(err, errors.IssueLink{ + IssueURL: "https://github.com/cockroachdb/cockroach/issues/67765", + Detail: "you have encountered a known bug in CockroachDB, please consider " + + "reporting on the Github issue or reach out via Support. " + + "This assertion can be disabled by setting the environment variable " + + "COCKROACH_DISABLE_COMMIT_SANITY_CHECK=true", + })) + if !DisableCommitSanityCheck { + log.Fatalf(ctx, "%s", err) + } + return roachpb.NewError(err) } // setTxnAnchorKey sets the key at which to anchor the transaction record. The