Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
72483: kv: RangeKeyMismatchError cleanup. r=nvanbenschoten a=mneverov

kv: RangeKeyMismatchError cleanup.

Fixes #72399

72848: server: only sleep to ensure clock monotonicity on server restart r=nvanbenschoten a=nvanbenschoten

This was broken in #72276 in an effort to fix a previous regression.
It recovers the previous behavior of only sleeping when a server is
being restarted, but doing so even if a max HLC upper bound has not
been persisted.

This fixes a regression in the speed of logic tests:
```
// before
ok  	github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder	53.891s

// after
ok  	github.com/cockroachdb/cockroach/pkg/sql/opt/exec/execbuilder	40.891s
```

72888: pg_catalog: check UndefinedTable error code when looking up table r=otan a=rafiss

I don't actually know how to get to this code path, but I'm adding this
just in case as a companion to ac6415a
which I added on release-21.2.

Release note: None

72892: pg_catalog: fix pg_indexes so it respects intervalstyle r=otan a=rafiss

Release note (bug fix): Previously, index definitions in
pg_catalog.pg_indexes would not format intervals according to the
intervalstyle session variable. This is fixed now.

Co-authored-by: Max Neverov <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
4 people committed Nov 18, 2021
5 parents 945da18 + e0d97ca + 315c349 + 25c4a90 + e05fe66 commit 1a07867
Show file tree
Hide file tree
Showing 10 changed files with 305 additions and 379 deletions.
6 changes: 5 additions & 1 deletion pkg/kv/bulk/sst_batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -466,7 +466,11 @@ func AddSSTable(
if m := (*roachpb.RangeKeyMismatchError)(nil); errors.As(err, &m) {
// TODO(andrei): We just use the first of m.Ranges; presumably we
// should be using all of them to avoid further retries.
split := m.Ranges()[0].Desc.EndKey.AsRawKey()
mr, err := m.MismatchedRange()
if err != nil {
return err
}
split := mr.Desc.EndKey.AsRawKey()
log.Infof(ctx, "SSTable cannot be added spanning range bounds %v, retrying...", split)
left, right, err := createSplitSSTable(ctx, db, item.start, split, item.disallowShadowing, iter, settings)
if err != nil {
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -1643,7 +1643,7 @@ func (ds *DistSender) sendPartialBatch(
// Range descriptor might be out of date - evict it. This is likely the
// result of a range split. If we have new range descriptors, insert them
// instead.
for _, ri := range tErr.Ranges() {
for _, ri := range tErr.Ranges {
// Sanity check that we got the different descriptors. Getting the same
// descriptor and putting it in the cache would be bad, as we'd go through
// an infinite loops of retries.
Expand All @@ -1653,7 +1653,7 @@ func (ds *DistSender) sendPartialBatch(
routingTok.Desc(), ri.Desc, pErr))}
}
}
routingTok.EvictAndReplace(ctx, tErr.Ranges()...)
routingTok.EvictAndReplace(ctx, tErr.Ranges...)
// On addressing errors (likely a split), we need to re-invoke
// the range descriptor lookup machinery, so we recurse by
// sending batch to just the partial span this descriptor was
Expand All @@ -1662,7 +1662,7 @@ func (ds *DistSender) sendPartialBatch(
// to it matches the positions into our batch (using the full
// batch here would give a potentially larger response slice
// with unknown mapping to our truncated reply).
log.VEventf(ctx, 1, "likely split; will resend. Got new descriptors: %s", tErr.Ranges())
log.VEventf(ctx, 1, "likely split; will resend. Got new descriptors: %s", tErr.Ranges)
reply, pErr = ds.divideAndSendBatchToRanges(ctx, ba, rs, withCommit, batchIdx)
return response{reply: reply, positions: positions, pErr: pErr}
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -858,9 +858,9 @@ func TestReplicaRangeBoundsChecking(t *testing.T) {
if mismatchErr, ok := pErr.GetDetail().(*roachpb.RangeKeyMismatchError); !ok {
t.Errorf("expected range key mismatch error: %s", pErr)
} else {
require.Len(t, mismatchErr.Ranges(), 2)
mismatchedDesc := mismatchErr.Ranges()[0].Desc
suggestedDesc := mismatchErr.Ranges()[1].Desc
require.Len(t, mismatchErr.Ranges, 2)
mismatchedDesc := mismatchErr.Ranges[0].Desc
suggestedDesc := mismatchErr.Ranges[1].Desc
if mismatchedDesc.RangeID != firstRepl.RangeID {
t.Errorf("expected mismatched range to be %d, found %v", firstRepl.RangeID, mismatchedDesc)
}
Expand Down
5 changes: 4 additions & 1 deletion pkg/kv/kvserver/store_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,10 @@ func (s *Store) Send(
// We'll return info on the range that the request ended up being routed to
// and, to the extent that we have the info, the ranges containing the keys
// that the client requested, and all the ranges in between.
ri := t.Ranges()[0]
ri, err := t.MismatchedRange()
if err != nil {
return nil, roachpb.NewError(err)
}
skipRID := ri.Desc.RangeID // We already have info on one range, so don't add it again below.
startKey := ri.Desc.StartKey
if rSpan.Key.Less(startKey) {
Expand Down
36 changes: 16 additions & 20 deletions pkg/roachpb/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -574,9 +574,8 @@ func NewRangeKeyMismatchError(
}
}
e := &RangeKeyMismatchError{
RequestStartKey: start,
RequestEndKey: end,
DeprecatedMismatchedRange: *desc,
RequestStartKey: start,
RequestEndKey: end,
}
// More ranges are sometimes added to rangesInternal later.
e.AppendRangeInfo(ctx, *desc, l)
Expand All @@ -588,31 +587,28 @@ func (e *RangeKeyMismatchError) Error() string {
}

func (e *RangeKeyMismatchError) message(_ *Error) string {
desc := &e.Ranges()[0].Desc
mr, err := e.MismatchedRange()
if err != nil {
return err.Error()
}
return fmt.Sprintf("key range %s-%s outside of bounds of range %s-%s; suggested ranges: %s",
e.RequestStartKey, e.RequestEndKey, desc.StartKey, desc.EndKey, e.Ranges())
e.RequestStartKey, e.RequestEndKey, mr.Desc.StartKey, mr.Desc.EndKey, e.Ranges)
}

// Type is part of the ErrorDetailInterface.
func (e *RangeKeyMismatchError) Type() ErrorDetailType {
return RangeKeyMismatchErrType
}

// Ranges returns the range info for the range that the request was erroneously
// routed to. It deals with legacy errors coming from 20.1 nodes by returning
// empty lease for the respective descriptors.
//
// At least one RangeInfo is returned.
func (e *RangeKeyMismatchError) Ranges() []RangeInfo {
if len(e.rangesInternal) != 0 {
return e.rangesInternal
}
// Fallback for 20.1 errors. Remove in 21.1.
ranges := []RangeInfo{{Desc: e.DeprecatedMismatchedRange}}
if e.DeprecatedSuggestedRange != nil {
ranges = append(ranges, RangeInfo{Desc: *e.DeprecatedSuggestedRange})
// MismatchedRange returns the range info for the range that the request was
// erroneously routed to, or an error if the Ranges slice is empty.
func (e *RangeKeyMismatchError) MismatchedRange() (RangeInfo, error) {
if len(e.Ranges) == 0 {
return RangeInfo{}, errors.AssertionFailedf(
"RangeKeyMismatchError (key range %s-%s) with empty RangeInfo slice", e.RequestStartKey, e.RequestEndKey,
)
}
return ranges
return e.Ranges[0], nil
}

// AppendRangeInfo appends info about one range to the set returned to the
Expand All @@ -628,7 +624,7 @@ func (e *RangeKeyMismatchError) AppendRangeInfo(
log.Fatalf(ctx, "lease names missing replica; lease: %s, desc: %s", l, desc)
}
}
e.rangesInternal = append(e.rangesInternal, RangeInfo{
e.Ranges = append(e.Ranges, RangeInfo{
Desc: desc,
Lease: l,
})
Expand Down
Loading

0 comments on commit 1a07867

Please sign in to comment.