Skip to content

Commit 6e31b79

Browse files
committed
storage: process intents synchronously
Process intents synchronously on the goroutine which generated them.
1 parent aecff9f commit 6e31b79

File tree

4 files changed

+28
-84
lines changed

4 files changed

+28
-84
lines changed

pkg/storage/intent_resolver.go

Lines changed: 2 additions & 59 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ import (
2424
"github.com/cockroachdb/cockroach/pkg/roachpb"
2525
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
2626
"github.com/cockroachdb/cockroach/pkg/util/log"
27-
"github.com/cockroachdb/cockroach/pkg/util/stop"
2827
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
2928
"github.com/cockroachdb/cockroach/pkg/util/tracing"
3029
"github.com/cockroachdb/cockroach/pkg/util/uuid"
@@ -39,11 +38,6 @@ import (
3938
// TODO(bdarnell): how to determine best value?
4039
const intentResolverTaskLimit = 100
4140

42-
type resolveTask struct {
43-
r *Replica
44-
intents []intentsWithArg
45-
}
46-
4741
// intentResolver manages the process of pushing transactions and
4842
// resolving intents.
4943
type intentResolver struct {
@@ -56,48 +50,17 @@ type intentResolver struct {
5650
// Maps transaction ids to a refcount.
5751
inFlight map[uuid.UUID]int
5852
}
59-
60-
resolveWorkCh chan struct{}
61-
resolveWorkMu struct {
62-
syncutil.Mutex
63-
tasks []resolveTask
64-
}
6553
}
6654

6755
func newIntentResolver(store *Store) *intentResolver {
6856
ir := &intentResolver{
69-
store: store,
70-
sem: make(chan struct{}, intentResolverTaskLimit),
71-
resolveWorkCh: make(chan struct{}, 1),
57+
store: store,
58+
sem: make(chan struct{}, intentResolverTaskLimit),
7259
}
7360
ir.mu.inFlight = map[uuid.UUID]int{}
7461
return ir
7562
}
7663

77-
func (ir *intentResolver) Start(stopper *stop.Stopper) {
78-
stopper.RunWorker(func() {
79-
for {
80-
select {
81-
case <-ir.resolveWorkCh:
82-
for {
83-
ir.resolveWorkMu.Lock()
84-
tasks := ir.resolveWorkMu.tasks
85-
ir.resolveWorkMu.tasks = nil
86-
ir.resolveWorkMu.Unlock()
87-
if len(tasks) == 0 {
88-
break
89-
}
90-
for _, task := range tasks {
91-
ir.processIntents(task.r, task.intents)
92-
}
93-
}
94-
case <-stopper.ShouldStop():
95-
return
96-
}
97-
}
98-
})
99-
}
100-
10164
// processWriteIntentError tries to push the conflicting
10265
// transaction(s) responsible for the given WriteIntentError, and to
10366
// resolve those intents if possible. Returns a new error to be used
@@ -302,26 +265,6 @@ func (ir *intentResolver) maybePushTransactions(
302265
// differently and would be better served by different entry points,
303266
// but combining them simplifies the plumbing necessary in Replica.
304267
func (ir *intentResolver) processIntentsAsync(r *Replica, intents []intentsWithArg) {
305-
if len(intents) == 0 {
306-
return
307-
}
308-
ir.resolveWorkMu.Lock()
309-
ir.resolveWorkMu.tasks = append(ir.resolveWorkMu.tasks, resolveTask{r, intents})
310-
ir.resolveWorkMu.Unlock()
311-
select {
312-
case ir.resolveWorkCh <- struct{}{}:
313-
default:
314-
}
315-
}
316-
317-
// processIntents asynchronously processes intents which were
318-
// encountered during another command but did not interfere with the
319-
// execution of that command. This occurs in two cases: inconsistent
320-
// reads and EndTransaction (which queues its own external intents for
321-
// processing via this method). The two cases are handled somewhat
322-
// differently and would be better served by different entry points,
323-
// but combining them simplifies the plumbing necessary in Replica.
324-
func (ir *intentResolver) processIntents(r *Replica, intents []intentsWithArg) {
325268
now := r.store.Clock().Now()
326269
ctx := context.TODO()
327270
stopper := r.store.Stopper()

pkg/storage/replica.go

Lines changed: 17 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -183,6 +183,7 @@ type proposalResult struct {
183183
Reply *roachpb.BatchResponse
184184
Err *roachpb.Error
185185
ProposalRetry proposalRetryReason
186+
Intents []intentsWithArg
186187
}
187188

188189
type replicaChecksum struct {
@@ -1664,10 +1665,9 @@ func (r *Replica) addReadOnlyCmd(
16641665
// described in #2231.
16651666
pErr = r.checkIfTxnAborted(ctx, r.store.Engine(), *ba.Txn)
16661667
}
1667-
if result.Local.intents != nil && len(*result.Local.intents) > 0 {
1668-
log.Eventf(ctx, "submitting %d intents to asynchronous processing",
1669-
len(*result.Local.intents))
1670-
r.store.intentResolver.processIntentsAsync(r, *result.Local.intents)
1668+
if intents := result.Local.detachIntents(); len(intents) > 0 {
1669+
log.Eventf(ctx, "submitting %d intents to asynchronous processing", len(intents))
1670+
r.store.intentResolver.processIntentsAsync(r, intents)
16711671
}
16721672
if pErr != nil {
16731673
log.ErrEvent(ctx, pErr.String())
@@ -1869,6 +1869,14 @@ func (r *Replica) tryAddWriteCmd(
18691869
// Set endCmds to nil because they have already been invoked
18701870
// in processRaftCommand.
18711871
endCmds = nil
1872+
if len(propResult.Intents) > 0 {
1873+
// Semi-synchronously process any intents that need resolving here in
1874+
// order to apply back pressure on the client which generated them. The
1875+
// resolution is semi-synchronous in that there is a limited number of
1876+
// outstanding asynchronous resolution tasks allowed after which
1877+
// further calls will block.
1878+
r.store.intentResolver.processIntentsAsync(r, propResult.Intents)
1879+
}
18721880
return propResult.Reply, propResult.Err, propResult.ProposalRetry
18731881
case <-ctxDone:
18741882
// If our context was cancelled, return an AmbiguousResultError
@@ -2077,12 +2085,13 @@ func (r *Replica) propose(
20772085
// An error here corresponds to a failfast-proposal: The command resulted
20782086
// in an error and did not need to commit a batch (the common error case).
20792087
if pErr != nil {
2088+
intents := pCmd.Local.detachIntents()
20802089
r.handleEvalResult(ctx, repDesc, pCmd.Local, pCmd.Replicated)
20812090
if endCmds != nil {
20822091
endCmds.done(nil, pErr, proposalNoRetry)
20832092
}
20842093
ch := make(chan proposalResult, 1)
2085-
ch <- proposalResult{Err: pErr}
2094+
ch <- proposalResult{Err: pErr, Intents: intents}
20862095
close(ch)
20872096
return ch, func() bool { return false }, nil
20882097
}
@@ -3287,6 +3296,7 @@ func (r *Replica) processRaftCommand(
32873296
} else {
32883297
log.Fatalf(ctx, "proposal must return either a reply or an error: %+v", cmd)
32893298
}
3299+
response.Intents = cmd.Local.detachIntents()
32903300
lResult = cmd.Local
32913301
}
32923302

@@ -4224,11 +4234,11 @@ func (r *Replica) loadSystemConfigSpan() ([]roachpb.KeyValue, []byte, error) {
42244234
if pErr != nil {
42254235
return nil, nil, pErr.GoError()
42264236
}
4227-
if result.Local.intents != nil && len(*result.Local.intents) > 0 {
4237+
if intents := result.Local.detachIntents(); len(intents) > 0 {
42284238
// There were intents, so what we read may not be consistent. Attempt
42294239
// to nudge the intents in case they're expired; next time around we'll
42304240
// hopefully have more luck.
4231-
r.store.intentResolver.processIntentsAsync(r, *result.Local.intents)
4241+
r.store.intentResolver.processIntentsAsync(r, intents)
42324242
return nil, nil, errSystemConfigIntent
42334243
}
42344244
kvs := br.Responses[0].GetInner().(*roachpb.ScanResponse).Rows

pkg/storage/replica_proposal.go

Lines changed: 9 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,15 @@ func (lResult *LocalEvalResult) finish(pr proposalResult) {
110110
close(lResult.doneCh)
111111
}
112112

113+
func (lResult *LocalEvalResult) detachIntents() []intentsWithArg {
114+
if lResult.intents == nil {
115+
return nil
116+
}
117+
intents := *lResult.intents
118+
lResult.intents = nil
119+
return intents
120+
}
121+
113122
// EvalResult is the result of evaluating a KV request. That is, the
114123
// proposer (which holds the lease, at least in the case in which the command
115124
// will complete successfully) has evaluated the request and is holding on to:
@@ -600,22 +609,6 @@ func (r *Replica) handleLocalEvalResult(
600609
// Non-state updates and actions.
601610
// ======================
602611

603-
if originReplica.StoreID == r.store.StoreID() {
604-
// On the replica on which this command originated, resolve skipped
605-
// intents asynchronously - even on failure.
606-
//
607-
// TODO(tschottdorf): EndTransaction will use this pathway to return
608-
// intents which should immediately be resolved. However, there's
609-
// a slight chance that an error between the origin of that intents
610-
// slice and here still results in that intent slice arriving here
611-
// without the EndTransaction having committed. We should clearly
612-
// separate the part of the EvalResult which also applies on errors.
613-
if lResult.intents != nil {
614-
r.store.intentResolver.processIntentsAsync(r, *lResult.intents)
615-
}
616-
}
617-
lResult.intents = nil
618-
619612
// The above are present too often, so we assert only if there are
620613
// "nontrivial" actions below.
621614
shouldAssert = (lResult != LocalEvalResult{})

pkg/storage/store.go

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -992,8 +992,6 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error {
992992
// Add the bookie to the store.
993993
s.bookie = newBookie(s.metrics)
994994

995-
s.intentResolver.Start(stopper)
996-
997995
// Read the store ident if not already initialized. "NodeID != 0" implies
998996
// the store has already been initialized.
999997
if s.Ident.NodeID == 0 {

0 commit comments

Comments
 (0)