Skip to content

Commit

Permalink
kv: RangeKeyMismatchError cleanup.
Browse files Browse the repository at this point in the history
Fixes #72399

Release note: None
  • Loading branch information
mneverov committed Nov 5, 2021
1 parent f84b8e7 commit a3c860c
Show file tree
Hide file tree
Showing 9 changed files with 233 additions and 373 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/bulk/sst_batcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -460,7 +460,7 @@ 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()
split := m.Ranges[0].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 @@ -1636,7 +1636,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 @@ -1646,7 +1646,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 @@ -1655,7 +1655,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
4 changes: 3 additions & 1 deletion pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1422,7 +1422,9 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) {
{err: &roachpb.TransactionPushError{}, asyncAbort: false},
{err: &roachpb.TransactionRetryError{}, asyncAbort: false},
{err: &roachpb.RangeNotFoundError{}, asyncAbort: false},
{err: &roachpb.RangeKeyMismatchError{}, asyncAbort: false},
{err: &roachpb.RangeKeyMismatchError{
Ranges: []roachpb.RangeInfo{{}}}, asyncAbort: false,
},
{err: &roachpb.TransactionStatusError{}, asyncAbort: false},
}

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 @@ -918,9 +918,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
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/store_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ 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 := t.Ranges[0]
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
3 changes: 1 addition & 2 deletions pkg/kv/txn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,8 +259,7 @@ func TestRunTransactionRetryOnErrors(t *testing.T) {
{&roachpb.TransactionPushError{}, true},
{&roachpb.TransactionRetryError{}, true},
{&roachpb.WriteTooOldError{}, true},
{&roachpb.RangeNotFoundError{}, false},
{&roachpb.RangeKeyMismatchError{}, false},
{&roachpb.RangeKeyMismatchError{Ranges: []roachpb.RangeInfo{{}}}, false},
{&roachpb.TransactionStatusError{}, false},
}

Expand Down
28 changes: 5 additions & 23 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,33 +587,16 @@ func (e *RangeKeyMismatchError) Error() string {
}

func (e *RangeKeyMismatchError) message(_ *Error) string {
desc := &e.Ranges()[0].Desc
desc := &e.Ranges[0].Desc
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, desc.StartKey, 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})
}
return ranges
}

// AppendRangeInfo appends info about one range to the set returned to the
// kvclient.
//
Expand All @@ -628,7 +610,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 a3c860c

Please sign in to comment.