Skip to content

Commit

Permalink
storage: add "noop" intent resolution poisoning option
Browse files Browse the repository at this point in the history
Manual testing in cockroachdb#15997 surfaced that one limiting
factor in resolving many intents is contention on the transaction's abort cache entry. In one
extreme test, I wrote 10E6 abortable intents into a single range, in which case the GC queue sends
very large batches of intent resolution requests for the same transaction to the intent resolver.

These requests all overlapped on the transaction's abort cache key, causing very slow progress, and
ultimately preventing the GC queue from making a dent in the minute allotted to it. Generally this
appears to be a somewhat atypical case, but since @nvanbenschoten observed something similar in
cockroachdb#18199 it seemed well worth addressing, by means of

1. allow intent resolutions to not touch the abort span
2. correctly declare the keys for `ResolveIntent{,Range}` to only declare the abort cache key
   if it is actually going to be accessed.

With these changes, the gc queue was able to clear out a million intents comfortably on my older
13" MacBook (single node).

Also use this option in the intent resolver, where possible -- most transactions don't receive abort
cache entries, and intents are often "found" by multiple conflicting writers. We want to avoid
adding artificial contention there, though in many situations the same intent is resolved and so a
conflict still exists.

Migration: a new field number was added to the proto and the old one preserved. We continue to
populate it. Downstream of Raft, we use the new field but if it's unset, synthesize from the
deprecated field. I believe this is sufficient and we can just remove all traces of the old field in
v1.3. (v1.1 uses the old, v1.2 uses the new with compatibility for the old, v1.3 only the new field).
  • Loading branch information
tbg committed Sep 30, 2017
1 parent 081f0ba commit 66507ae
Show file tree
Hide file tree
Showing 8 changed files with 633 additions and 339 deletions.
711 changes: 409 additions & 302 deletions pkg/roachpb/api.pb.go

Large diffs are not rendered by default.

22 changes: 20 additions & 2 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -604,6 +604,18 @@ message QueryTxnResponse {
repeated bytes waiting_txns = 3 [(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID"];
}


enum PoisonType {
// Sender is at an older version and uses the deprecated_poison field.
Deprecated = 0;
// Make no changes to the abort cache of the associated transaction.
Noop = 1;
// Poison the abort cache of the associated transaction.
Do = 2;
// Clear any entry in the abort cache of the associated transaction.
Clear = 3;
}

// A ResolveIntentRequest is arguments to the ResolveIntent()
// method. It is sent by transaction coordinators after success
// calling PushTxn to clean up write intents: either to remove, commit
Expand All @@ -618,7 +630,10 @@ message ResolveIntentRequest {
optional TransactionStatus status = 3 [(gogoproto.nullable) = false];
// Optionally poison the sequence cache for the transaction the intent's
// range.
optional bool poison = 4 [(gogoproto.nullable) = false];
//
// TODO(tschottdorf): this can be removed in CockroachDB v1.3.
optional bool deprecated_poison = 4 [(gogoproto.nullable) = false];
optional PoisonType poison = 5 [(gogoproto.nullable) = false];;
}

// A ResolveIntentResponse is the return value from the
Expand All @@ -641,7 +656,10 @@ message ResolveIntentRangeRequest {
optional TransactionStatus status = 3 [(gogoproto.nullable) = false];
// Optionally poison the sequence cache for the transaction on all ranges
// on which the intents reside.
optional bool poison = 4 [(gogoproto.nullable) = false];
//
// TODO(tschottdorf): this can be removed in CockroachDB v1.3.
optional bool deprecated_poison = 4 [(gogoproto.nullable) = false];
optional PoisonType poison = 5 [(gogoproto.nullable) = false];
}

// A NoopResponse is the return value from a no-op operation.
Expand Down
15 changes: 12 additions & 3 deletions pkg/storage/gc_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -408,8 +408,9 @@ func processLocalKeyRange(
if err := func() error {
infoMu.Unlock() // intentional
defer infoMu.Lock()
// This transaction has committed, so poisoning is not relevant.
return resolveIntents(roachpb.AsIntents(txn.Intents, &txn),
ResolveOptions{Wait: true, Poison: false})
ResolveOptions{Wait: true, Poison: roachpb.PoisonType_Noop})
}(); err != nil {
// Ignore above error, but if context is expired, no point in keeping going.
if ctx.Err() != nil {
Expand All @@ -424,7 +425,8 @@ func processLocalKeyRange(
if err := func() error {
infoMu.Unlock() // intentional
defer infoMu.Lock()
return resolveIntents(roachpb.AsIntents(txn.Intents, &txn), ResolveOptions{Wait: true, Poison: false})
// Contention between transactions is not a problem in this path, so use Clear.
return resolveIntents(roachpb.AsIntents(txn.Intents, &txn), ResolveOptions{Wait: true, Poison: roachpb.PoisonType_Clear})
}(); err != nil {
// Returning the error here would abort the whole GC run, and
// we don't want that. Instead, we simply don't GC this entry.
Expand Down Expand Up @@ -898,7 +900,14 @@ func RunGC(
}
}

if err := resolveIntentsFn(intents, ResolveOptions{Wait: true, Poison: false}); err != nil {
// Note that we resolve with PoisonType_NOOP, meaning "leave the abort cache alone". This is
// helpful because it causes less command queue overlap between the various batches the intent
// resolver partitions our intents into (when many of the intents belong to the same
// transaction). The abort cache is cleared out below, so we're not missing anything major here.
//
// TODO(tschottdorf): the intent resolver *could* be smarter here and remember what it already
// cleared. Doubt it's worth it.
if err := resolveIntentsFn(intents, ResolveOptions{Wait: true, Poison: roachpb.PoisonType_Noop}); err != nil {
return nil, GCInfo{}, err
}

Expand Down
45 changes: 30 additions & 15 deletions pkg/storage/intent_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,8 +105,16 @@ func (ir *intentResolver) processWriteIntentError(
return pErr
}

// If we're pushing the other transaction's timestamp, we don't need to touch the abort cache.
// If we abort, we do. Note that we never clear out the cache here, which would be dangerous as
// it could let an aborted-but-running transaction access keys that formerly held its intents.
poisonType := roachpb.PoisonType_Noop
if pushType == roachpb.PUSH_ABORT {
poisonType = roachpb.PoisonType_Do
}

if err := ir.resolveIntents(ctx, resolveIntents,
ResolveOptions{Wait: false, Poison: pushType == roachpb.PUSH_ABORT}); err != nil {
ResolveOptions{Wait: false, Poison: poisonType}); err != nil {
return roachpb.NewError(err)
}

Expand Down Expand Up @@ -322,9 +330,10 @@ func (ir *intentResolver) processIntents(
// time of writing has our push abort it, leading to the
// same situation as above.
//
// Thus, we must poison.
// Thus, we must poison (but note that this only has effects
// for those intents which are actually aborted).
if err := ir.resolveIntents(ctxWithTimeout, resolveIntents,
ResolveOptions{Wait: true, Poison: true}); err != nil {
ResolveOptions{Wait: true, Poison: roachpb.PoisonType_Do}); err != nil {
log.Warningf(ctx, "%s: failed to resolve intents: %s", r, err)
return
}
Expand All @@ -334,15 +343,17 @@ func (ir *intentResolver) processIntents(
}
} else { // EndTransaction
// For EndTransaction, we know the transaction is finalized so
// we can skip the push and go straight to the resolve.
// we can skip the push and go straight to the resolve, clearing
// out the abort cache (for intents that are ABORTED, so for a
// successful commit, none).
//
// This mechanism assumes that when an EndTransaction fails,
// the client makes no assumptions about the result. For
// example, an attempt to explicitly rollback the transaction
// may succeed (triggering this code path), but the result may
// not make it back to the client.
if err := ir.resolveIntents(ctxWithTimeout, item.intents,
ResolveOptions{Wait: true, Poison: false}); err != nil {
ResolveOptions{Wait: true, Poison: roachpb.PoisonType_Clear}); err != nil {
log.Warningf(ctx, "%s: failed to resolve intents: %s", r, err)
return
}
Expand All @@ -365,7 +376,7 @@ func (ir *intentResolver) processIntents(
// at least KeyLocalMax.
//
// #7880 will address this by making GCRequest less special and
// thus obviating the need to cook up an artificial range here.
// thus obviates the need to cook up an artificial range here.
var gcArgs roachpb.GCRequest
{
key := keys.MustAddr(txn.Key)
Expand Down Expand Up @@ -396,7 +407,7 @@ func (ir *intentResolver) processIntents(
// call to block, and whether the ranges containing the intents are to be poisoned.
type ResolveOptions struct {
Wait bool
Poison bool
Poison roachpb.PoisonType
}

// resolveIntents resolves the given intents. `wait` is currently a
Expand Down Expand Up @@ -428,24 +439,28 @@ func (ir *intentResolver) resolveIntents(
defer cleanup()
log.Eventf(ctx, "resolving intents [wait=%t]", opts.Wait)

var deprecatedPoison = opts.Poison == roachpb.PoisonType_Do

var reqs []roachpb.Request
for i := range intents {
intent := intents[i] // avoids a race in `i, intent := range ...`
var resolveArgs roachpb.Request
{
if len(intent.EndKey) == 0 {
resolveArgs = &roachpb.ResolveIntentRequest{
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
Poison: opts.Poison,
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
DeprecatedPoison: deprecatedPoison,
Poison: opts.Poison,
}
} else {
resolveArgs = &roachpb.ResolveIntentRangeRequest{
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
Poison: opts.Poison,
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
DeprecatedPoison: deprecatedPoison,
Poison: opts.Poison,
}
}
}
Expand Down
67 changes: 51 additions & 16 deletions pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -1807,25 +1807,52 @@ func setAbortCache(
batch engine.ReadWriter,
ms *enginepb.MVCCStats,
txn enginepb.TxnMeta,
poison bool,
poisonType roachpb.PoisonType,
) error {
if !poison {
switch poisonType {
case roachpb.PoisonType_Clear:
return rec.AbortCache().Del(ctx, batch, ms, txn.ID)
case roachpb.PoisonType_Do:
entry := roachpb.AbortCacheEntry{
Key: txn.Key,
Timestamp: txn.Timestamp,
Priority: txn.Priority,
}
return rec.AbortCache().Put(ctx, batch, ms, txn.ID, &entry)
case roachpb.PoisonType_Noop:
return nil
default:
return errors.Errorf("unhandled PoisonType: %d", poisonType)
}
entry := roachpb.AbortCacheEntry{
Key: txn.Key,
Timestamp: txn.Timestamp,
Priority: txn.Priority,
}
return rec.AbortCache().Put(ctx, batch, ms, txn.ID, &entry)
}

func writesAbortCache(intentStatus roachpb.TransactionStatus, poisonType roachpb.PoisonType) bool {
return (intentStatus == roachpb.ABORTED && poisonType == roachpb.PoisonType_Do) ||
poisonType == roachpb.PoisonType_Clear

}

func declareKeysResolveIntent(
desc roachpb.RangeDescriptor, header roachpb.Header, req roachpb.Request, spans *SpanSet,
) {
DefaultDeclareKeys(desc, header, req, spans)
ri := req.(*roachpb.ResolveIntentRequest)
spans.Add(SpanReadWrite, roachpb.Span{Key: keys.AbortCacheKey(header.RangeID, ri.IntentTxn.ID)})

if writesAbortCache(ri.Status, ri.Poison) {
spans.Add(SpanReadWrite, roachpb.Span{Key: keys.AbortCacheKey(header.RangeID, ri.IntentTxn.ID)})
}
}

func fromDeprecatedPoison(poison roachpb.PoisonType, deprecatedPoison bool) roachpb.PoisonType {
if poison != roachpb.PoisonType_Deprecated {
return poison
}
if deprecatedPoison {
poison = roachpb.PoisonType_Do
} else {
poison = roachpb.PoisonType_Clear
}
return poison
}

// evalResolveIntent resolves a write intent from the specified key
Expand All @@ -1834,23 +1861,27 @@ func evalResolveIntent(
ctx context.Context, batch engine.ReadWriter, cArgs CommandArgs, resp roachpb.Response,
) (EvalResult, error) {
args := cArgs.Args.(*roachpb.ResolveIntentRequest)
poison := fromDeprecatedPoison(args.Poison, args.DeprecatedPoison)
intent := roachpb.Intent{
Span: args.Span,
Txn: args.IntentTxn,
Status: args.Status,
}
args = nil // avoid accidental use below

h := cArgs.Header

ms := cArgs.Stats

if h.Txn != nil {
return EvalResult{}, errTransactionUnsupported
}

intent := roachpb.Intent{
Span: args.Span,
Txn: args.IntentTxn,
Status: args.Status,
}
if err := engine.MVCCResolveWriteIntent(ctx, batch, ms, intent); err != nil {
return EvalResult{}, err
}
if intent.Status == roachpb.ABORTED {
return EvalResult{}, setAbortCache(ctx, cArgs.EvalCtx, batch, ms, args.IntentTxn, args.Poison)
return EvalResult{}, setAbortCache(ctx, cArgs.EvalCtx, batch, ms, intent.Txn, poison)
}
return EvalResult{}, nil
}
Expand All @@ -1860,7 +1891,9 @@ func declareKeysResolveIntentRange(
) {
DefaultDeclareKeys(desc, header, req, spans)
ri := req.(*roachpb.ResolveIntentRangeRequest)
spans.Add(SpanReadWrite, roachpb.Span{Key: keys.AbortCacheKey(header.RangeID, ri.IntentTxn.ID)})
if writesAbortCache(ri.Status, ri.Poison) {
spans.Add(SpanReadWrite, roachpb.Span{Key: keys.AbortCacheKey(header.RangeID, ri.IntentTxn.ID)})
}
}

// evalResolveIntentRange resolves write intents in the specified
Expand All @@ -1869,6 +1902,8 @@ func evalResolveIntentRange(
ctx context.Context, batch engine.ReadWriter, cArgs CommandArgs, resp roachpb.Response,
) (EvalResult, error) {
args := cArgs.Args.(*roachpb.ResolveIntentRangeRequest)
args.Poison = fromDeprecatedPoison(args.Poison, args.DeprecatedPoison)

h := cArgs.Header
ms := cArgs.Stats

Expand Down
90 changes: 90 additions & 0 deletions pkg/storage/replica_command_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// Copyright 2017 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package storage

import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

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

txnMeta := enginepb.TxnMeta{}
desc := roachpb.RangeDescriptor{
RangeID: 99,
StartKey: roachpb.RKey("a"),
EndKey: roachpb.RKey("a"),
}
for _, test := range []struct {
status roachpb.TransactionStatus
poison roachpb.PoisonType
expSpans []string
}{
{
status: roachpb.ABORTED,
poison: roachpb.PoisonType_Noop,
expSpans: []string{"1 0: {b-c}"},
},
{
status: roachpb.ABORTED,
poison: roachpb.PoisonType_Clear,
expSpans: []string{"1 0: {b-c}", `1 1: /{Local/RangeID/99/r/AbortCache/"00000000-0000-0000-0000-000000000000"-Min}`},
},
{
status: roachpb.ABORTED,
poison: roachpb.PoisonType_Do,
expSpans: []string{"1 0: {b-c}", `1 1: /{Local/RangeID/99/r/AbortCache/"00000000-0000-0000-0000-000000000000"-Min}`},
},
{
status: roachpb.COMMITTED,
poison: roachpb.PoisonType_Noop,
expSpans: []string{"1 0: {b-c}"},
},
{
status: roachpb.COMMITTED,
poison: roachpb.PoisonType_Clear,
expSpans: []string{"1 0: {b-c}", `1 1: /{Local/RangeID/99/r/AbortCache/"00000000-0000-0000-0000-000000000000"-Min}`},
},
{
status: roachpb.COMMITTED,
poison: roachpb.PoisonType_Do,
expSpans: []string{"1 0: {b-c}"},
},
} {
t.Run("", func(t *testing.T) {
ri := roachpb.ResolveIntentRequest{
IntentTxn: txnMeta,
Status: test.status,
Poison: test.poison,
}
ri.Key = roachpb.Key("b")
ri.EndKey = roachpb.Key("c")

var spans SpanSet
var h roachpb.Header
h.RangeID = desc.RangeID
declareKeysResolveIntent(desc, h, &ri, &spans)
exp := strings.Join(test.expSpans, "\n")
if s := strings.TrimSpace(spans.String()); s != exp {
t.Errorf("expected %s, got %s", exp, s)
}
})
}
}
Loading

0 comments on commit 66507ae

Please sign in to comment.