Skip to content

Commit

Permalink
Merge #73603
Browse files Browse the repository at this point in the history
73603: kvcoord: make "txn unexpectedly committed" assertion opt-out r=nvanbenschoten a=tbg

We are currently investigating how we are hitting this assertion,
but while it fires it is doing more harm than good.

See #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


Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Dec 13, 2021
2 parents 976cc79 + 903e2b3 commit 64f55ce
Show file tree
Hide file tree
Showing 3 changed files with 149 additions and 9 deletions.
6 changes: 6 additions & 0 deletions pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ go_library(
"//pkg/util",
"//pkg/util/contextutil",
"//pkg/util/ctxgroup",
"//pkg/util/envutil",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
Expand Down Expand Up @@ -118,6 +119,7 @@ go_test(
"integration_test.go",
"main_test.go",
"range_iter_test.go",
"replayed_commit_test.go",
"replica_slice_test.go",
"send_test.go",
"split_test.go",
Expand Down Expand Up @@ -148,6 +150,7 @@ go_test(
"//pkg/kv/kvbase",
"//pkg/kv/kvclient/rangecache:with-mocks",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/batcheval",
"//pkg/kv/kvserver/closedts",
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/kv/kvserver/kvserverbase",
Expand All @@ -170,8 +173,10 @@ go_test(
"//pkg/testutils/localtestcluster",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/caller",
"//pkg/util/grpcutil",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand All @@ -184,6 +189,7 @@ go_test(
"//pkg/util/syncutil",
"//pkg/util/tracing",
"//pkg/util/uuid",
"@com_github_cockroachdb_circuitbreaker//:circuitbreaker",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//errutil",
Expand Down
110 changes: 110 additions & 0 deletions pkg/kv/kvclient/kvcoord/replayed_commit_test.go
Original file line number Diff line number Diff line change
@@ -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)
}
42 changes: 33 additions & 9 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"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"
Expand All @@ -34,6 +35,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
Expand Down Expand Up @@ -910,7 +917,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)
}
Expand All @@ -922,22 +929,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
Expand Down

0 comments on commit 64f55ce

Please sign in to comment.