diff --git a/pkg/kv/kvserver/batcheval/declare_test.go b/pkg/kv/kvserver/batcheval/declare_test.go index 270bc3fda82a..93b340555751 100644 --- a/pkg/kv/kvserver/batcheval/declare_test.go +++ b/pkg/kv/kvserver/batcheval/declare_test.go @@ -85,7 +85,8 @@ func TestRequestsSerializeWithAllKeys(t *testing.T) { if err != nil { t.Error(err) } - if !allLatchSpans.Intersects(&otherLatchSpans) { + checker := spanset.NewSpanChecker(&allLatchSpans) + if !checker.Intersects(&otherLatchSpans) { t.Errorf("%s does not serialize with declareAllKeys", method) } }) diff --git a/pkg/kv/kvserver/replica_eval_context.go b/pkg/kv/kvserver/replica_eval_context.go index e66f4d235cf1..7f991026ed8b 100644 --- a/pkg/kv/kvserver/replica_eval_context.go +++ b/pkg/kv/kvserver/replica_eval_context.go @@ -109,8 +109,8 @@ func NewReplicaEvalContext( rec = newEvalContextImpl(ctx, r, requiresClosedTSOlderThanStorageSnap, ah) if util.RaceEnabled { return &SpanSetReplicaEvalContext{ - i: rec, - ss: *ss, + i: rec, + checker: *spanset.NewSpanChecker(ss), } } return rec diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 58266346161b..7198540384e7 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -29,8 +29,8 @@ import ( // ReplicaEvalContext which verifies that access to state is registered in the // SpanSet if one is given. type SpanSetReplicaEvalContext struct { - i batcheval.EvalContext - ss spanset.SpanSet + i batcheval.EvalContext + checker spanset.SpanChecker } var _ batcheval.EvalContext = &SpanSetReplicaEvalContext{} @@ -103,7 +103,7 @@ func (rec *SpanSetReplicaEvalContext) IsFirstRange() bool { // Desc returns the Replica's RangeDescriptor. func (rec SpanSetReplicaEvalContext) Desc() *roachpb.RangeDescriptor { desc := rec.i.Desc() - rec.ss.AssertAllowed(spanset.SpanReadOnly, + rec.checker.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)}, ) return desc @@ -144,7 +144,7 @@ func (rec SpanSetReplicaEvalContext) GetMaxSplitCPU(ctx context.Context) (float6 func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( ctx context.Context, txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, ) (bool, kvpb.TransactionAbortedReason) { - rec.ss.AssertAllowed(spanset.SpanReadOnly, + rec.checker.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) return rec.i.CanCreateTxnRecord(ctx, txnID, txnKey, txnMinTS) @@ -156,7 +156,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( func (rec SpanSetReplicaEvalContext) MinTxnCommitTS( ctx context.Context, txnID uuid.UUID, txnKey []byte, ) hlc.Timestamp { - rec.ss.AssertAllowed(spanset.SpanReadOnly, + rec.checker.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) return rec.i.MinTxnCommitTS(ctx, txnID, txnKey) @@ -166,7 +166,7 @@ func (rec SpanSetReplicaEvalContext) MinTxnCommitTS( // keys are garbage collected. Reads and writes at timestamps <= this time will // not be served. func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp { - rec.ss.AssertAllowed(spanset.SpanReadOnly, + rec.checker.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())}, ) return rec.i.GetGCThreshold() @@ -190,7 +190,7 @@ func (rec SpanSetReplicaEvalContext) String() string { func (rec SpanSetReplicaEvalContext) GetLastReplicaGCTimestamp( ctx context.Context, ) (hlc.Timestamp, error) { - if err := rec.ss.CheckAllowed(spanset.SpanReadOnly, + if err := rec.checker.CheckAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLastReplicaGCTimestampKey(rec.GetRangeID())}, ); err != nil { return hlc.Timestamp{}, err @@ -222,11 +222,11 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context) // To capture a read summary over the range, all keys must be latched for // writing to prevent any concurrent reads or writes. desc := rec.i.Desc() - rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + rec.checker.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ Key: keys.MakeRangeKeyPrefix(desc.StartKey), EndKey: keys.MakeRangeKeyPrefix(desc.EndKey), }) - rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + rec.checker.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), }) diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index f7f18e423720..5950dadd45e3 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -2722,6 +2722,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) { 0, ) require.NoError(t, err) + checker := spanset.NewSpanChecker(&spans) for _, tc := range []struct { access spanset.SpanAccess key roachpb.Key @@ -2732,7 +2733,7 @@ func TestReplicaLatchingSplitDeclaresWrites(t *testing.T) { {spanset.SpanReadWrite, roachpb.Key("b"), false}, {spanset.SpanReadWrite, roachpb.Key("d"), true}, } { - err := spans.CheckAllowed(tc.access, roachpb.Span{Key: tc.key}) + err := checker.CheckAllowed(tc.access, roachpb.Span{Key: tc.key}) if tc.expectAccess { require.NoError(t, err) } else { @@ -5344,7 +5345,11 @@ func TestAbortSpanError(t *testing.T) { } ec := newEvalContextImpl(ctx, tc.repl, false /* requireClosedTS */, kvpb.AdmissionHeader{}) - rec := &SpanSetReplicaEvalContext{ec, *allSpans()} + ss := allSpans() + rec := &SpanSetReplicaEvalContext{ + i: ec, + checker: *spanset.NewSpanChecker(ss), + } pErr := checkIfTxnAborted(ctx, rec, tc.engine, txn) if _, ok := pErr.GetDetail().(*kvpb.TransactionAbortedError); ok { expected := txn.Clone() diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 10a9f294e814..0a54c568480e 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -23,8 +23,8 @@ import ( // MVCCIterator wraps an storage.MVCCIterator and ensures that it can // only be used to access spans in a SpanSet. type MVCCIterator struct { - i storage.MVCCIterator - spans *SpanSet + i storage.MVCCIterator + checker *SpanChecker // spansOnly controls whether or not timestamps associated with the // spans are considered when ensuring access. If set to true, @@ -48,13 +48,13 @@ var _ storage.MVCCIterator = &MVCCIterator{} // iterator against the given SpanSet. Timestamps associated with the spans // in the spanset are not considered, only the span boundaries are checked. func NewIterator(iter storage.MVCCIterator, spans *SpanSet) *MVCCIterator { - return &MVCCIterator{i: iter, spans: spans, spansOnly: true} + return &MVCCIterator{i: iter, checker: NewSpanChecker(spans), spansOnly: true} } // NewIteratorAt constructs an iterator that verifies access of the underlying // iterator against the given SpanSet at the given timestamp. func NewIteratorAt(iter storage.MVCCIterator, spans *SpanSet, ts hlc.Timestamp) *MVCCIterator { - return &MVCCIterator{i: iter, spans: spans, ts: ts} + return &MVCCIterator{i: iter, checker: NewSpanChecker(spans), ts: ts} } // Close is part of the storage.MVCCIterator interface. @@ -137,9 +137,9 @@ func (i *MVCCIterator) checkAllowed(span roachpb.Span, errIfDisallowed bool) { func (i *MVCCIterator) checkAllowedValidPos(span roachpb.Span, errIfDisallowed bool) { var err error if i.spansOnly { - err = i.spans.CheckAllowed(SpanReadOnly, span) + err = i.checker.CheckAllowed(SpanReadOnly, span) } else { - err = i.spans.CheckAllowedAt(SpanReadOnly, span, i.ts) + err = i.checker.CheckAllowedAt(SpanReadOnly, span, i.ts) } if errIfDisallowed { i.err = err @@ -213,11 +213,11 @@ func (i *MVCCIterator) FindSplitKey( start, end, minSplitKey roachpb.Key, targetSize int64, ) (storage.MVCCKey, error) { if i.spansOnly { - if err := i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { return storage.MVCCKey{}, err } } else { - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, i.ts); err != nil { + if err := i.checker.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, i.ts); err != nil { return storage.MVCCKey{}, err } } @@ -242,8 +242,9 @@ func (i *MVCCIterator) UnsafeLazyValue() pebble.LazyValue { // EngineIterator wraps a storage.EngineIterator and ensures that it can // only be used to access spans in a SpanSet. type EngineIterator struct { - i storage.EngineIterator - spans *SpanSet + i storage.EngineIterator + //spans *SpanSet + checker *SpanChecker spansOnly bool ts hlc.Timestamp } @@ -261,10 +262,10 @@ func (i *EngineIterator) SeekEngineKeyGE(key storage.EngineKey) (valid bool, err } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.checker.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { return false, err } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + } else if err = i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { return false, err } return valid, err @@ -278,10 +279,10 @@ func (i *EngineIterator) SeekEngineKeyLT(key storage.EngineKey) (valid bool, err } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.checker.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { return false, err } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { + } else if err = i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { return false, err } return valid, err @@ -313,7 +314,7 @@ func (i *EngineIterator) SeekEngineKeyGEWithLimit( if state != pebble.IterValid { return state, err } - if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + if err = i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { return pebble.IterExhausted, err } return state, err @@ -327,7 +328,7 @@ func (i *EngineIterator) SeekEngineKeyLTWithLimit( if state != pebble.IterValid { return state, err } - if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { + if err = i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{EndKey: key.Key}); err != nil { return pebble.IterExhausted, err } return state, err @@ -354,11 +355,11 @@ func (i *EngineIterator) checkKeyAllowed() (valid bool, err error) { } if key.IsMVCCKey() && !i.spansOnly { mvccKey, _ := key.ToMVCCKey() - if err := i.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { + if err := i.checker.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: mvccKey.Key}, i.ts); err != nil { // Invalid, but no error. return false, nil // nolint:returnerrcheck } - } else if err = i.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { + } else if err = i.checker.CheckAllowed(SpanReadOnly, roachpb.Span{Key: key.Key}); err != nil { // Invalid, but no error. return false, nil // nolint:returnerrcheck } @@ -431,8 +432,8 @@ func (i *EngineIterator) Stats() storage.IteratorStats { } type spanSetReader struct { - r storage.Reader - spans *SpanSet + r storage.Reader + checker *SpanChecker spansOnly bool ts hlc.Timestamp @@ -469,11 +470,11 @@ func (s spanSetReader) MVCCIterate( f func(storage.MVCCKeyValue, storage.MVCCRangeKeyStack) error, ) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := s.checker.CheckAllowed(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { + if err := s.checker.CheckAllowedAt(SpanReadOnly, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { return err } } @@ -488,9 +489,9 @@ func (s spanSetReader) NewMVCCIterator( return nil, err } if s.spansOnly { - return NewIterator(mvccIter, s.spans), nil + return NewIterator(mvccIter, s.checker.spans), nil } - return NewIteratorAt(mvccIter, s.spans, s.ts), nil + return NewIteratorAt(mvccIter, s.checker.spans, s.ts), nil } func (s spanSetReader) NewEngineIterator( @@ -501,8 +502,9 @@ func (s spanSetReader) NewEngineIterator( return nil, err } return &EngineIterator{ - i: engineIter, - spans: s.spans, + i: engineIter, + //spans: s.checker.spans, + checker: s.checker, spansOnly: s.spansOnly, ts: s.ts, }, nil @@ -519,8 +521,8 @@ func (s spanSetReader) PinEngineStateForIterators(readCategory fs.ReadCategory) } type spanSetWriter struct { - w storage.Writer - spans *SpanSet + w storage.Writer + checker *SpanChecker spansOnly bool ts hlc.Timestamp @@ -535,11 +537,11 @@ func (s spanSetWriter) ApplyBatchRepr(repr []byte, sync bool) error { func (s spanSetWriter) checkAllowed(key roachpb.Key) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key}); err != nil { + if err := s.checker.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key}, s.ts); err != nil { + if err := s.checker.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key}, s.ts); err != nil { return err } } @@ -561,7 +563,7 @@ func (s spanSetWriter) ClearUnversioned(key roachpb.Key, opts storage.ClearOptio } func (s spanSetWriter) ClearEngineKey(key storage.EngineKey, opts storage.ClearOptions) error { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.checker.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { return err } return s.w.ClearEngineKey(key, opts) @@ -575,11 +577,11 @@ func (s spanSetWriter) SingleClearEngineKey(key storage.EngineKey) error { func (s spanSetWriter) checkAllowedRange(start, end roachpb.Key) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}); err != nil { + if err := s.checker.CheckAllowed(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { + if err := s.checker.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: start, EndKey: end}, s.ts); err != nil { return err } } @@ -661,11 +663,11 @@ func (s spanSetWriter) ClearMVCCRangeKey(rangeKey storage.MVCCRangeKey) error { func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error { if s.spansOnly { - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.checker.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { return err } } else { - if err := s.spans.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key.Key}, s.ts); err != nil { + if err := s.checker.CheckAllowedAt(SpanReadWrite, roachpb.Span{Key: key.Key}, s.ts); err != nil { return err } } @@ -697,7 +699,7 @@ func (s spanSetWriter) PutEngineKey(key storage.EngineKey, value []byte) error { if !s.spansOnly { panic("cannot do timestamp checking for putting EngineKey") } - if err := s.spans.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { + if err := s.checker.CheckAllowed(SpanReadWrite, roachpb.Span{Key: key.Key}); err != nil { return err } return s.w.PutEngineKey(key, value) @@ -731,17 +733,19 @@ var _ storage.ReadWriter = ReadWriter{} func makeSpanSetReadWriter(rw storage.ReadWriter, spans *SpanSet) ReadWriter { spans = addLockTableSpans(spans) + checker := NewSpanChecker(spans) return ReadWriter{ - spanSetReader: spanSetReader{r: rw, spans: spans, spansOnly: true}, - spanSetWriter: spanSetWriter{w: rw, spans: spans, spansOnly: true}, + spanSetReader: spanSetReader{r: rw, checker: checker, spansOnly: true}, + spanSetWriter: spanSetWriter{w: rw, checker: checker, spansOnly: true}, } } func makeSpanSetReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Timestamp) ReadWriter { spans = addLockTableSpans(spans) + checker := NewSpanChecker(spans) return ReadWriter{ - spanSetReader: spanSetReader{r: rw, spans: spans, ts: ts}, - spanSetWriter: spanSetWriter{w: rw, spans: spans, ts: ts}, + spanSetReader: spanSetReader{r: rw, checker: checker, ts: ts}, + spanSetWriter: spanSetWriter{w: rw, checker: checker, ts: ts}, } } @@ -752,7 +756,7 @@ func makeSpanSetReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Times // NewReader clones and does not retain the provided span set. func NewReader(r storage.Reader, spans *SpanSet, ts hlc.Timestamp) storage.Reader { spans = addLockTableSpans(spans) - return spanSetReader{r: r, spans: spans, ts: ts} + return spanSetReader{r: r, checker: NewSpanChecker(spans), ts: ts} } // NewReadWriterAt returns a storage.ReadWriter that asserts access of the diff --git a/pkg/kv/kvserver/spanset/spanset.go b/pkg/kv/kvserver/spanset/spanset.go index f9df171dd2ab..4013cabb7f21 100644 --- a/pkg/kv/kvserver/spanset/spanset.go +++ b/pkg/kv/kvserver/spanset/spanset.go @@ -208,7 +208,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() } @@ -232,17 +231,76 @@ func (s *SpanSet) GetSpans(access SpanAccess, scope SpanScope) []Span { return s.spans[access][scope] } +// Validate returns an error if any spans that have been added to the set +// are invalid. +func (s *SpanSet) Validate() error { + for sa := SpanAccess(0); sa < NumSpanAccess; sa++ { + for ss := SpanScope(0); ss < NumSpanScope; ss++ { + for _, cur := range s.GetSpans(sa, ss) { + if len(cur.EndKey) > 0 && cur.Key.Compare(cur.EndKey) >= 0 { + return errors.Errorf("inverted span %s %s", cur.Key, cur.EndKey) + } + } + } + } + + 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). +func (s *SpanSet) DisableUndeclaredAccessAssertions() { + s.allowUndeclared = true +} + +// contains returns whether s1 contains s2. Unlike Span.Contains, this function +// supports spans with a nil start key and a non-nil end key (e.g. "[nil, c)"). +// In this form, s2.Key (inclusive) is considered to be the previous key to +// s2.EndKey (exclusive). +func contains(s1, s2 roachpb.Span) bool { + if s2.Key != nil { + // The common case. + return s1.Contains(s2) + } + + // The following is equivalent to: + // s1.Contains(roachpb.Span{Key: s2.EndKey.Prev()}) + + if s1.EndKey == nil { + return s1.Key.IsPrev(s2.EndKey) + } + + return s1.Key.Compare(s2.EndKey) < 0 && s1.EndKey.Compare(s2.EndKey) >= 0 +} + +// SpanChecker wraps a SpanSet and provides verification methods for checking +// whether access to spans is allowed. This separates the responsibility of +// representing a set of spans (SpanSet) from verifying access to them +// (SpanChecker). +type SpanChecker struct { + spans *SpanSet +} + +// NewSpanChecker creates a new SpanChecker that verifies access against the +// given SpanSet. +func NewSpanChecker(spans *SpanSet) *SpanChecker { + return &SpanChecker{ + spans: spans, + } +} + // Intersects returns true iff the span set denoted by `other` has any -// overlapping spans with `s`, and that those spans overlap in access type. Note -// that timestamps associated with the spans in the spanset are not considered, -// only the span boundaries are checked. -func (s *SpanSet) Intersects(other *SpanSet) bool { +// overlapping spans with the wrapped spanset, and that those spans overlap in +// access type. Note that timestamps associated with the spans in the spanset +// are not considered, only the span boundaries are checked. +func (c *SpanChecker) Intersects(other *SpanSet) bool { for sa := SpanAccess(0); sa < NumSpanAccess; sa++ { for ss := SpanScope(0); ss < NumSpanScope; ss++ { otherSpans := other.GetSpans(sa, ss) for _, span := range otherSpans { // If access is allowed, we must have an overlap. - if err := s.CheckAllowed(sa, span.Span); err == nil { + if err := c.CheckAllowed(sa, span.Span); err == nil { return true } } @@ -254,8 +312,8 @@ func (s *SpanSet) Intersects(other *SpanSet) bool { // AssertAllowed calls CheckAllowed and fatals if the access is not allowed. // Timestamps associated with the spans in the spanset are not considered, // only the span boundaries are checked. -func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) { - if err := s.CheckAllowed(access, span); err != nil { +func (c *SpanChecker) AssertAllowed(access SpanAccess, span roachpb.Span) { + if err := c.CheckAllowed(access, span); err != nil { log.KvExec.Fatalf(context.TODO(), "%v", err) } } @@ -275,19 +333,19 @@ func (s *SpanSet) AssertAllowed(access SpanAccess, span roachpb.Span) { // fail at checking if read only access over the span [a-d) was requested. This // is also a problem if the added spans were read only and the spanset wasn't // already SortAndDedup-ed. -func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error { - return s.checkAllowed(access, span, func(_ SpanAccess, _ Span) bool { +func (c *SpanChecker) CheckAllowed(access SpanAccess, span roachpb.Span) error { + return c.checkAllowed(access, span, func(_ SpanAccess, _ Span) bool { return true }) } // CheckAllowedAt is like CheckAllowed, except it returns an error if the access // is not allowed over the given keyspan at the given timestamp. -func (s *SpanSet) CheckAllowedAt( +func (c *SpanChecker) CheckAllowedAt( access SpanAccess, span roachpb.Span, timestamp hlc.Timestamp, ) error { mvcc := !timestamp.IsEmpty() - return s.checkAllowed(access, span, func(declAccess SpanAccess, declSpan Span) bool { + return c.checkAllowed(access, span, func(declAccess SpanAccess, declSpan Span) bool { declTimestamp := declSpan.Timestamp if declTimestamp.IsEmpty() { // When the span is declared as non-MVCC (i.e. with an empty @@ -328,10 +386,10 @@ func (s *SpanSet) CheckAllowedAt( }) } -func (s *SpanSet) checkAllowed( +func (c *SpanChecker) checkAllowed( access SpanAccess, span roachpb.Span, check func(SpanAccess, Span) bool, ) error { - if s.allowUndeclared { + if c.spans.allowUndeclared { // If the request has specified that undeclared spans are allowed, do // nothing. return nil @@ -344,55 +402,12 @@ func (s *SpanSet) checkAllowed( } for ac := access; ac < NumSpanAccess; ac++ { - for _, cur := range s.spans[ac][scope] { + for _, cur := range c.spans.spans[ac][scope] { if contains(cur.Span, span) && check(ac, cur) { return nil } } } - return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, s, debugutil.Stack()) -} - -// contains returns whether s1 contains s2. Unlike Span.Contains, this function -// supports spans with a nil start key and a non-nil end key (e.g. "[nil, c)"). -// In this form, s2.Key (inclusive) is considered to be the previous key to -// s2.EndKey (exclusive). -func contains(s1, s2 roachpb.Span) bool { - if s2.Key != nil { - // The common case. - return s1.Contains(s2) - } - - // The following is equivalent to: - // s1.Contains(roachpb.Span{Key: s2.EndKey.Prev()}) - - if s1.EndKey == nil { - return s1.Key.IsPrev(s2.EndKey) - } - - return s1.Key.Compare(s2.EndKey) < 0 && s1.EndKey.Compare(s2.EndKey) >= 0 -} - -// Validate returns an error if any spans that have been added to the set -// are invalid. -func (s *SpanSet) Validate() error { - for sa := SpanAccess(0); sa < NumSpanAccess; sa++ { - for ss := SpanScope(0); ss < NumSpanScope; ss++ { - for _, cur := range s.GetSpans(sa, ss) { - if len(cur.EndKey) > 0 && cur.Key.Compare(cur.EndKey) >= 0 { - return errors.Errorf("inverted span %s %s", cur.Key, cur.EndKey) - } - } - } - } - - 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). -func (s *SpanSet) DisableUndeclaredAccessAssertions() { - s.allowUndeclared = true + return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, c.spans, debugutil.Stack()) } diff --git a/pkg/kv/kvserver/spanset/spanset_test.go b/pkg/kv/kvserver/spanset/spanset_test.go index f58c2d65b046..ef6c3c366e30 100644 --- a/pkg/kv/kvserver/spanset/spanset_test.go +++ b/pkg/kv/kvserver/spanset/spanset_test.go @@ -155,8 +155,9 @@ func TestSpanSetCheckAllowedBoundaries(t *testing.T) { {Key: roachpb.Key("c"), EndKey: roachpb.Key("d")}, {Key: roachpb.Key("l"), EndKey: roachpb.Key("m")}, } + checker := NewSpanChecker(&bdGkq) for _, span := range allowed { - if err := bdGkq.CheckAllowed(SpanReadOnly, span); err != nil { + if err := checker.CheckAllowed(SpanReadOnly, span); err != nil { t.Errorf("expected %s to be allowed, but got error: %+v", span, err) } } @@ -185,7 +186,7 @@ func TestSpanSetCheckAllowedBoundaries(t *testing.T) { {Key: roachpb.Key("k"), EndKey: roachpb.Key("q").Next()}, } for _, span := range disallowed { - if err := bdGkq.CheckAllowed(SpanReadOnly, span); err == nil { + if err := checker.CheckAllowed(SpanReadOnly, span); err == nil { t.Errorf("expected %s to be disallowed", span) } } @@ -202,6 +203,8 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { ss.AddMVCC(SpanReadWrite, roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2}) ss.AddNonMVCC(SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(1)}) + checker := NewSpanChecker(&ss) + var allowedRO = []struct { span roachpb.Span ts hlc.Timestamp @@ -224,7 +227,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { {roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{WallTime: 1}}, } for _, tc := range allowedRO { - if err := ss.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); err != nil { + if err := checker.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); err != nil { t.Errorf("expected %s at %s to be allowed, but got error: %+v", tc.span, tc.ts, err) } } @@ -255,7 +258,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { {roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}}, } for _, tc := range allowedRW { - if err := ss.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); err != nil { + if err := checker.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); err != nil { t.Errorf("expected %s at %s to be allowed, but got error: %+v", tc.span, tc.ts, err) } } @@ -273,7 +276,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { {roachpb.Span{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 3}}, } for _, tc := range disallowedRO { - if err := ss.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); !testutils.IsError(err, readErr) { + if err := checker.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); !testutils.IsError(err, readErr) { t.Errorf("expected %s at %s to be disallowed", tc.span, tc.ts) } } @@ -299,7 +302,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) { {roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 1}}, } for _, tc := range disallowedRW { - if err := ss.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); !testutils.IsError(err, writeErr) { + if err := checker.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); !testutils.IsError(err, writeErr) { t.Errorf("expected %s at %s to be disallowed", tc.span, tc.ts) } } @@ -313,13 +316,15 @@ func TestSpanSetCheckAllowedReversed(t *testing.T) { bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}) bdGkq.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("k"), EndKey: roachpb.Key("q")}) + checker := NewSpanChecker(&bdGkq) + allowed := []roachpb.Span{ // Exactly as declared. {EndKey: roachpb.Key("d")}, {EndKey: roachpb.Key("q")}, } for _, span := range allowed { - if err := bdGkq.CheckAllowed(SpanReadOnly, span); err != nil { + if err := checker.CheckAllowed(SpanReadOnly, span); err != nil { t.Errorf("expected %s to be allowed, but got error: %+v", span, err) } } @@ -331,7 +336,7 @@ func TestSpanSetCheckAllowedReversed(t *testing.T) { {EndKey: roachpb.Key("k")}, } for _, span := range disallowed { - if err := bdGkq.CheckAllowed(SpanReadOnly, span); err == nil { + if err := checker.CheckAllowed(SpanReadOnly, span); err == nil { t.Errorf("expected %s to be disallowed", span) } } @@ -346,13 +351,15 @@ func TestSpanSetCheckAllowedAtReversed(t *testing.T) { bdGkq.AddMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}, ts) bdGkq.AddMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("k"), EndKey: roachpb.Key("q")}, ts) + checker := NewSpanChecker(&bdGkq) + allowed := []roachpb.Span{ // Exactly as declared. {EndKey: roachpb.Key("d")}, {EndKey: roachpb.Key("q")}, } for _, span := range allowed { - if err := bdGkq.CheckAllowedAt(SpanReadOnly, span, ts); err != nil { + if err := checker.CheckAllowedAt(SpanReadOnly, span, ts); err != nil { t.Errorf("expected %s to be allowed, but got error: %+v", span, err) } } @@ -364,7 +371,7 @@ func TestSpanSetCheckAllowedAtReversed(t *testing.T) { {EndKey: roachpb.Key("k")}, } for _, span := range disallowed { - if err := bdGkq.CheckAllowedAt(SpanReadOnly, span, ts); err == nil { + if err := checker.CheckAllowedAt(SpanReadOnly, span, ts); err == nil { t.Errorf("expected %s to be disallowed", span) } } @@ -381,16 +388,18 @@ func TestSpanSetWriteImpliesRead(t *testing.T) { ss.AddNonMVCC(SpanReadOnly, roSpan) ss.AddNonMVCC(SpanReadWrite, rwSpan) - if err := ss.CheckAllowed(SpanReadOnly, roSpan); err != nil { + checker := NewSpanChecker(&ss) + + if err := checker.CheckAllowed(SpanReadOnly, roSpan); err != nil { t.Errorf("expected to be allowed to read roSpan, error: %+v", err) } - if err := ss.CheckAllowed(SpanReadWrite, roSpan); err == nil { + if err := checker.CheckAllowed(SpanReadWrite, roSpan); err == nil { t.Errorf("expected not to be allowed to write roSpan") } - if err := ss.CheckAllowed(SpanReadOnly, rwSpan); err != nil { + if err := checker.CheckAllowed(SpanReadOnly, rwSpan); err != nil { t.Errorf("expected to be allowed to read rwSpan, error: %+v", err) } - if err := ss.CheckAllowed(SpanReadWrite, rwSpan); err != nil { + if err := checker.CheckAllowed(SpanReadWrite, rwSpan); err != nil { t.Errorf("expected to be allowed to read rwSpan, error: %+v", err) } }