-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
intent_resolver.go
351 lines (329 loc) · 13.5 KB
/
intent_resolver.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
// Copyright 2016 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. See the AUTHORS file
// for names of contributors.
//
// Author: Ben Darnell
package storage
import (
"fmt"
"sync"
"github.com/cockroachdb/cockroach/base"
"github.com/cockroachdb/cockroach/client"
"github.com/cockroachdb/cockroach/keys"
"github.com/cockroachdb/cockroach/roachpb"
"github.com/cockroachdb/cockroach/util"
"github.com/cockroachdb/cockroach/util/log"
"github.com/cockroachdb/cockroach/util/tracing"
"github.com/opentracing/opentracing-go"
"golang.org/x/net/context"
)
// intentResolver manages the process of pushing transactions and
// resolving intents.
type intentResolver struct {
store *Store
}
func newIntentResolver(store *Store) *intentResolver {
return &intentResolver{store}
}
// maybePushTransaction tries to push the conflicting transaction(s)
// responsible for the given WriteIntentError: either move its
// timestamp forward on a read/write conflict, abort it on a
// write/write conflict, or do nothing if the transaction is no longer
// pending.
//
// Returns a slice of intents which can now be resolved and an error
// which should be returned to the client in place of the original
// WriteIntentError. The returned intents should be resolved via
// intentResolver.resolveIntents regardless of any error returned by
// maybePushTransaction.
//
// The returned error may be a copy of the original WriteIntentError,
// with or without the Resolved flag set, which governs the client's
// retry behavior. (if the transaction is pushed, the Resolved flag is
// set to tell the client to retry immediately; otherwise it is false
// to cause the client to back off).
//
// Callers are involved with
// a) conflict resolution for commands being executed at the Store with the
// client waiting,
// b) resolving intents encountered during inconsistent operations, and
// c) resolving intents upon EndTransaction which are not local to the given
// range. This is the only path in which the transaction is going to be
// in non-pending state and doesn't require a push.
func (ir *intentResolver) maybePushTransactions(ctx context.Context, wiErr roachpb.WriteIntentError, args roachpb.Request, h roachpb.Header, pushType roachpb.PushTxnType) ([]roachpb.Intent, *roachpb.Error) {
method := args.Method()
pusherTxn := h.Txn
readOnly := roachpb.IsReadOnly(args) // TODO(tschottdorf): pass as param
args = nil
if log.V(6) {
log.Infoc(ctx, "resolving write intent %s", wiErr)
}
sp, cleanupSp := tracing.SpanFromContext(opStore, ir.store.Tracer(), ctx)
defer cleanupSp()
sp.LogEvent("intent resolution")
// Split intents into those we need to push and those which are good to
// resolve.
// TODO(tschottdorf): can optimize this and use same underlying slice.
var pushIntents, resolveIntents []roachpb.Intent
for _, intent := range wiErr.Intents {
// The current intent does not need conflict resolution.
if intent.Status != roachpb.PENDING {
resolveIntents = append(resolveIntents, intent)
} else {
pushIntents = append(pushIntents, intent)
}
}
// Attempt to push the transaction(s) which created the conflicting intent(s).
now := ir.store.Clock().Now()
// TODO(tschottdorf): need deduplication here (many pushes for the same
// txn are awkward but even worse, could ratchet up the priority).
// If there's no pusher, we communicate a priority by sending an empty
// txn with only the priority set.
if pusherTxn == nil {
pusherTxn = &roachpb.Transaction{
Priority: roachpb.MakePriority(h.UserPriority),
}
}
var pushReqs []roachpb.Request
for _, intent := range pushIntents {
pushReqs = append(pushReqs, &roachpb.PushTxnRequest{
Span: roachpb.Span{
Key: intent.Txn.Key,
},
PusherTxn: *pusherTxn,
PusheeTxn: intent.Txn,
PushTo: h.Timestamp,
// The timestamp is used by PushTxn for figuring out whether the
// transaction is abandoned. If we used the argument's timestamp
// here, we would run into busy loops because that timestamp
// usually stays fixed among retries, so it will never realize
// that a transaction has timed out. See #877.
Now: now,
PushType: pushType,
})
}
// TODO(kaneda): Set the transaction in the header so that the
// txn is correctly propagated in an error response.
b := &client.Batch{}
b.InternalAddRequest(pushReqs...)
br, pushErr := ir.store.db.RunWithResponse(b)
if pushErr != nil {
if log.V(1) {
log.Infoc(ctx, "on %s: %s", method, pushErr)
}
// For write/write conflicts within a transaction, propagate the
// push failure, not the original write intent error. The push
// failure will instruct the client to restart the transaction
// with a backoff.
if pusherTxn.ID != nil && !readOnly {
return nil, pushErr
}
// For read/write conflicts, return the write intent error which
// engages backoff/retry (with !Resolved). We don't need to
// restart the txn, only resend the read with a backoff.
return nil, roachpb.NewError(&wiErr)
}
wiErr.Resolved = true // success!
for i, intent := range pushIntents {
pushee := br.Responses[i].GetInner().(*roachpb.PushTxnResponse).PusheeTxn
intent.Txn = pushee.TxnMeta
intent.Status = pushee.Status
resolveIntents = append(resolveIntents, intent)
}
return resolveIntents, roachpb.NewError(&wiErr)
}
// processIntentsAsync asynchronously processes intents which were
// encountered during another command but did not interfere with the
// execution of that command. This occurs in two cases: inconsistent
// reads and EndTransaction (which queues its own intents for
// processing via this method).
func (ir *intentResolver) processIntentsAsync(r *Replica, intents []intentsWithArg) {
if len(intents) == 0 {
return
}
now := r.store.Clock().Now()
ctx := r.context()
stopper := r.store.Stopper()
for _, item := range intents {
// TODO(tschottdorf): avoid data race related to batch unrolling in ExecuteCmd;
// can probably go again when that provisional code there is gone. Should
// still be careful though, a retry could happen and race with args.
args := util.CloneProto(item.args).(roachpb.Request)
stopper.RunAsyncTask(func() {
// Everything here is best effort; give up rather than waiting
// too long (helps avoid deadlocks during test shutdown,
// although this is imperfect due to the use of an
// uninterruptible WaitGroup.Wait in beginCmds).
ctxWithTimeout, cancel := context.WithTimeout(ctx, base.NetworkTimeout)
defer cancel()
h := roachpb.Header{Timestamp: now}
resolveIntents, pushErr := ir.maybePushTransactions(ctxWithTimeout, roachpb.WriteIntentError{
Intents: item.intents,
}, args, h, roachpb.PUSH_TOUCH)
if pErr := ir.resolveIntents(ctxWithTimeout, r, resolveIntents, true /* wait */, false /* TODO(tschottdorf): #5088 */); pErr != nil {
log.Warningc(ctxWithTimeout, "failed to resolve intents: %s", pErr)
return
}
if wiErr, ok := pushErr.GetDetail().(*roachpb.WriteIntentError); !ok || wiErr == nil || !wiErr.Resolved {
log.Warningc(ctxWithTimeout, "failed to push during intent resolution: %s", pushErr)
return
}
// We successfully resolved the intents, so we're able to GC from
// the txn span directly. Note that the sequence cache was cleared
// out synchronously with EndTransaction (see comments within for
// an explanation of why that is kosher).
//
// Note that we poisoned the sequence caches on the external ranges
// above. This may seem counter-intuitive, but it's actually
// necessary: Assume a transaction has committed here, with two
// external intents, and assume that we did not poison. Normally,
// these two intents would be resolved in the same batch, but that
// is not guaranteed (for example, if DistSender has a stale
// descriptor after a Merge). When resolved separately, the first
// ResolveIntent would clear out the sequence cache; an individual
// write on the second (still present) intent could then be
// replayed and would resolve to a real value (at least for a
// window of time unless we delete the local txn entry). That's not
// OK for non-idempotent commands such as Increment.
// TODO(tschottdorf): We should have another side effect on
// MVCCResolveIntent (on commit/abort): If it were able to remove
// the txn from its corresponding entries in the timestamp cache,
// no more replays at the same timestamp would be possible. This
// appears to be a useful performance optimization; we could then
// not poison on EndTransaction. In fact, the above mechanism
// could be an effective alternative to sequence-cache based
// poisoning (or the whole sequence cache?) itself.
//
// TODO(tschottdorf): down the road, can probably unclog the system
// here by batching up a bunch of those GCRequests before proposing.
if args.Method() == roachpb.EndTransaction {
var ba roachpb.BatchRequest
txn := item.intents[0].Txn
gcArgs := roachpb.GCRequest{
Span: roachpb.Span{
Key: r.Desc().StartKey.AsRawKey(),
EndKey: r.Desc().EndKey.AsRawKey(),
},
}
gcArgs.Keys = append(gcArgs.Keys, roachpb.GCRequest_GCKey{Key: keys.TransactionKey(txn.Key, txn.ID)})
ba.Add(&gcArgs)
if _, pErr := r.addWriteCmd(ctxWithTimeout, ba, nil /* nil */); pErr != nil {
log.Warningf("could not GC completed transaction: %s", pErr)
}
}
})
}
}
// resolveIntents resolves the given intents. For those which are
// local to the range, we submit directly to the local Raft instance;
// all non-local intents are resolved asynchronously in a batch. If
// `wait` is true, all operations are carried out synchronously and an
// error is returned. Otherwise, the call returns without error as
// soon as all local resolve commands have been **proposed** (not
// executed). This ensures that if a waiting client retries
// immediately after calling this function, it will not hit the same
// intents again.
func (ir *intentResolver) resolveIntents(ctx context.Context, r *Replica, intents []roachpb.Intent, wait bool, poison bool) *roachpb.Error {
sp, cleanupSp := tracing.SpanFromContext(opReplica, ir.store.Tracer(), ctx)
defer cleanupSp()
ctx = opentracing.ContextWithSpan(ctx, nil) // we're doing async stuff below; those need new traces
sp.LogEvent(fmt.Sprintf("resolving intents [wait=%t]", wait))
var reqsRemote []roachpb.Request
baLocal := roachpb.BatchRequest{}
for i := range intents {
intent := intents[i] // avoids a race in `i, intent := range ...`
var resolveArgs roachpb.Request
var local bool // whether this intent lives on this Range
{
if len(intent.EndKey) == 0 {
resolveArgs = &roachpb.ResolveIntentRequest{
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
Poison: poison,
}
local = r.ContainsKey(intent.Key)
} else {
resolveArgs = &roachpb.ResolveIntentRangeRequest{
Span: intent.Span,
IntentTxn: intent.Txn,
Status: intent.Status,
Poison: poison,
}
local = r.ContainsKeyRange(intent.Key, intent.EndKey)
}
}
// If the intent isn't (completely) local, we'll need to send an external request.
// We'll batch them all up and send at the end.
if local {
baLocal.Add(resolveArgs)
} else {
reqsRemote = append(reqsRemote, resolveArgs)
}
}
// The local batch goes directly to Raft.
var wg sync.WaitGroup
if len(baLocal.Requests) > 0 {
action := func() *roachpb.Error {
// Trace this under the ID of the intent owner.
sp := r.store.Tracer().StartSpan("resolve intents")
defer sp.Finish()
ctx = opentracing.ContextWithSpan(ctx, sp)
// Always operate with a timeout when resolving intents: this
// prevents rare shutdown timeouts in tests.
ctxWithTimeout, cancel := context.WithTimeout(ctx, base.NetworkTimeout)
defer cancel()
_, pErr := r.addWriteCmd(ctxWithTimeout, baLocal, &wg)
return pErr
}
wg.Add(1)
if wait || !r.store.Stopper().RunAsyncTask(func() {
if err := action(); err != nil {
log.Warningf("unable to resolve local intents; %s", err)
}
}) {
// Still run the task when draining. Our caller already has a task and
// going async here again is merely for performance, but some intents
// need to be resolved because they might block other tasks. See #1684.
// Note that handleSkippedIntents has a TODO in case #1684 comes back.
if err := action(); err != nil {
return err
}
}
}
// Resolve all of the intents which aren't local to the Range.
if len(reqsRemote) > 0 {
b := &client.Batch{}
b.InternalAddRequest(reqsRemote...)
action := func() *roachpb.Error {
// TODO(tschottdorf): no tracing here yet.
return r.store.DB().Run(b)
}
if wait || !r.store.Stopper().RunAsyncTask(func() {
if err := action(); err != nil {
log.Warningf("unable to resolve external intents: %s", err)
}
}) {
// As with local intents, try async to not keep the caller waiting, but
// when draining just go ahead and do it synchronously. See #1684.
if err := action(); err != nil {
return err
}
}
}
// Wait until the local ResolveIntents batch has been submitted to
// raft. No-op if all were non-local.
wg.Wait()
return nil
}