Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 6 additions & 5 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,11 +76,6 @@ func declareKeysExport(
return err
}
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(header.RangeID)})
// Export requests will usually not hold latches during their evaluation.
//
// See call to `AssertAllowed()` in GetGCThreshold() to understand why we need
// to disable these assertions for export requests.
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

Expand All @@ -96,6 +91,12 @@ func evalExport(
ctx, evalExportSpan := tracing.ChildSpan(ctx, "evalExport")
defer evalExportSpan.Finish()

// Export requests will usually not hold latches during their evaluation.
//
// See call to `AssertAllowed()` in GetGCThreshold() to understand why we need
// to disable these assertions for export requests.
reader = spanset.DisableReaderAssertions(reader)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The problem is, we probably need to retain some control of which assertions are we disabling. For now, DisableReaderAssertions will unwrap the SpanSetBatch, which will disable all assertions really. Now, we only have one assertion type (Latch assertions), but we are soon planning to add a new assertion (Forbidden spans)


// Table's marked to be excluded from backup are expected to be configured
// with a short GC TTL. Additionally, backup excludes such table's from being
// protected from GC when writing ProtectedTimestamp records. The
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,6 @@ func declareKeysGC(
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
// Needed for updating optional GC hint.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeGCHintKey(rs.GetRangeID())})
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

Expand Down Expand Up @@ -139,6 +138,9 @@ func GC(
args := cArgs.Args.(*kvpb.GCRequest)
h := cArgs.Header

// TODO(pav-kv): find out why and comment.
readWriter = spanset.DisableReadWriterAssertions(readWriter)

// We do not allow GC requests to bump the GC threshold at the same time that
// they GC individual keys. This is because performing both of these actions
// at the same time could lead to a race where a read request is allowed to
Expand Down
8 changes: 5 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,18 +23,17 @@ func init() {
}

func declareKeysRequestLease(
rs ImmutableRangeState,
_ ImmutableRangeState,
_ *kvpb.Header,
_ kvpb.Request,
latchSpans *spanset.SpanSet,
_ *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) error {
// NOTE: RequestLease is run on replicas that do not hold the lease, so
// acquiring latches would not help synchronize with other requests. As
// such, the request does not declare latches. See also
// concurrency.shouldIgnoreLatches().
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

Expand All @@ -54,6 +53,9 @@ func RequestLease(
prevLease := args.PrevLease
newLease := args.Lease

// RequestLease does not hold latches by design.
readWriter = spanset.DisableReadWriterAssertions(readWriter)

// If this check is removed at some point, the filtering of learners on the
// sending side would have to be removed as well.
// TODO(nvanbenschoten): move this into leases.Verify.
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_recompute_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,6 @@ func declareKeysRecomputeStats(
rdKey := keys.RangeDescriptorKey(rs.GetStartKey())
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: rdKey})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(rdKey, uuid.Nil)})
// Disable the assertions which check that all reads were previously declared.
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

Expand All @@ -72,9 +70,12 @@ func RecomputeStats(
return result.Result{}, RecomputeStatsMismatchError
}
dryRun := args.DryRun

args = nil // avoid accidental use below

// Disable the assertions which check that all reads were previously declared.
// TODO(pav-kv): find out why we can do that.
reader = spanset.DisableReaderAssertions(reader)

actualMS, err := rditer.ComputeStatsForRange(ctx, desc, reader, cArgs.Header.Timestamp.WallTime)
if err != nil {
return result.Result{}, err
Expand Down
19 changes: 1 addition & 18 deletions pkg/kv/kvserver/spanset/spanset.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,7 @@ type Span struct {
// The Span slice for a particular access and scope contains non-overlapping
// spans in increasing key order after calls to SortAndDedup.
type SpanSet struct {
spans [NumSpanAccess][NumSpanScope][]Span
allowUndeclared bool
spans [NumSpanAccess][NumSpanScope][]Span
}

var spanSetPool = sync.Pool{
Expand Down Expand Up @@ -113,7 +112,6 @@ func (s *SpanSet) Release() {
s.spans[sa][ss] = recycle
}
}
s.allowUndeclared = false
spanSetPool.Put(s)
}

Expand Down Expand Up @@ -155,7 +153,6 @@ func (s *SpanSet) Copy() *SpanSet {
n.spans[sa][ss] = append(n.spans[sa][ss], s.spans[sa][ss]...)
}
}
n.allowUndeclared = s.allowUndeclared
return n
}

Expand Down Expand Up @@ -208,7 +205,6 @@ func (s *SpanSet) Merge(s2 *SpanSet) {
s.spans[sa][ss] = append(s.spans[sa][ss], s2.spans[sa][ss]...)
}
}
s.allowUndeclared = s2.allowUndeclared
s.SortAndDedup()
}

Expand Down Expand Up @@ -331,12 +327,6 @@ func (s *SpanSet) CheckAllowedAt(
func (s *SpanSet) checkAllowed(
access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool,
) error {
if s.allowUndeclared {
// If the request has specified that undeclared spans are allowed, do
// nothing.
return nil
}

scope := SpanGlobal
if (span.Key != nil && keys.IsLocal(span.Key)) ||
(span.EndKey != nil && keys.IsLocal(span.EndKey)) {
Expand Down Expand Up @@ -389,10 +379,3 @@ func (s *SpanSet) Validate() error {

return nil
}

// DisableUndeclaredAccessAssertions disables the assertions that prevent
// undeclared access to spans. This is generally set by requests that rely on
// other forms of synchronization for correctness (e.g. GCRequest).
Comment on lines -394 to -395
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"rely on other forms of synchronization" looks like a useful note, need to retain it in some form

func (s *SpanSet) DisableUndeclaredAccessAssertions() {
s.allowUndeclared = true
}
Loading