cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.11k stars 3.81k forks source link

kv: divergent ideas about whether a txn is "read-only" between the Txn and the txnIntentCollector #28256

Closed andreimatei closed 6 years ago

andreimatei commented 6 years ago

The intent collector interceptor complains if it receives an EndTransaction without it having collected any intents prior. It returns an error saying that it's a "readonly txn" and the EndTxn should have been elided above it. That error is supposed to signal bugs. https://github.com/cockroachdb/cockroach/blob/8044dea9c5c210c3ddb238a7edd446a93013e2e2/pkg/kv/txn_interceptor_intent_collector.go#L102

Unfortunately, that assumption is not quite copacetic with what the layers above it do. There's currently two problems:

  1. If the txn ever encountered an error, the Txn moves to the txnError error, at which point it forgets if it ever performed any writes (i.e. if it came from state txnReadonly or txnWriting, etc..), and so it will never elide future EndTransactions (rollbacks).
  2. If the BeginTxn batch is rejected below the Txn layer, in the TxnCoordSender, then the client.Txn considers the transaction to be writing (cause it sent a BeginTxn), but the intent collector considers it read-only (cause it never saw said BeginTxn). This happens for example if the Stopper is stopped when the BeginTransaction is sent, and the TCS fails to start the heartbeat loop and rejects the batch.

The first problem goes away in #28185 because that PR brakes apart the different txn states and correctly tracks if a BeginTxn was ever sent.

The second problem is more fundamental, caused by the separate tracking of the BeginTxn done in both the TCS and the intent collector. It's not very clear to me what to do about it. We could try to either share the "did we send a BeginTxn" state. This is a bit complicated by the fact that, in #28185, the BeginTxn tracking is not done by the TCS directly, but by the heartbeat interceptor. So we'd need to create a communication channel between two interceptors. I think what we'd want is for the txnLockGatekeeper to keep track of whether a BeginTxn is really sent to the server, and for the heartbeat interceptor to use that to dictate whether EndTransaction can be elided. Then any interceptor anywhere in the stack can retain the right to reject batches (as they tend to do already) and the intent collector can be left alone with the current assumptions - that if it sees and EndTxn it must have accumulated some intents. Alternatively, the intent collector could get its own logic for eliding the end transaction, duplicating the existing one. Yet alternatively, the intent collector could stop assuming anything and conservatively forward unnecessary EndTxns.

@nvanbenschoten for thoughts, if any. Thanks @jordanlewis for seeing something and saying something.

The error can be demonstrated with

func TestXXX(t *testing.T) {
    defer leaktest.AfterTest(t)()

    s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
    ctx := context.Background()
    s.Stopper().Stop(ctx)

    err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
        key := roachpb.Key("a")
        return txn.Put(ctx, key, "val")
    })
    if err != nil {
        t.Fatal(err)
    }
}
andreimatei commented 6 years ago

I've done something in #28185 - the gatekeeper is now in charge of tracking whether we ever sent a BeginTxn. Wasn't that bad.