Skip to content

Commit

Permalink
keys: Range takes a []RequestUnion, not BatchRequest
Browse files Browse the repository at this point in the history
The function didn't need the full batch header, so there was
not reason to pass the entire thing to it.

Release note: None
  • Loading branch information
nvanbenschoten committed Jul 15, 2019
1 parent d7232ed commit a18b131
Show file tree
Hide file tree
Showing 8 changed files with 20 additions and 21 deletions.
7 changes: 3 additions & 4 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -804,12 +804,11 @@ func EnsureSafeSplitKey(key roachpb.Key) (roachpb.Key, error) {
return key[:idx], nil
}

// Range returns a key range encompassing the key ranges of all requests in the
// Batch.
func Range(ba roachpb.BatchRequest) (roachpb.RSpan, error) {
// Range returns a key range encompassing the key ranges of all requests.
func Range(reqs []roachpb.RequestUnion) (roachpb.RSpan, error) {
from := roachpb.RKeyMax
to := roachpb.RKeyMin
for _, arg := range ba.Requests {
for _, arg := range reqs {
req := arg.GetInner()
h := req.Header()
if !roachpb.IsRange(req) && len(h.EndKey) != 0 {
Expand Down
6 changes: 3 additions & 3 deletions pkg/keys/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -474,7 +474,7 @@ func TestBatchRange(t *testing.T) {
Key: roachpb.Key(pair[0]), EndKey: roachpb.Key(pair[1]),
}})
}
if rs, err := Range(ba); err != nil {
if rs, err := Range(ba.Requests); err != nil {
t.Errorf("%d: %v", i, err)
} else if actPair := [2]string{string(rs.Key), string(rs.EndKey)}; !reflect.DeepEqual(actPair, c.exp) {
t.Errorf("%d: expected [%q,%q), got [%q,%q)", i, c.exp[0], c.exp[1], actPair[0], actPair[1])
Expand Down Expand Up @@ -503,7 +503,7 @@ func TestBatchError(t *testing.T) {
ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeader{
Key: roachpb.Key(c.req[0]), EndKey: roachpb.Key(c.req[1]),
}})
if _, err := Range(ba); !testutils.IsError(err, c.errMsg) {
if _, err := Range(ba.Requests); !testutils.IsError(err, c.errMsg) {
t.Errorf("%d: unexpected error %v", i, err)
}
}
Expand All @@ -513,7 +513,7 @@ func TestBatchError(t *testing.T) {
ba.Add(&roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{
Key: roachpb.Key("a"), EndKey: roachpb.Key("b"),
}})
if _, err := Range(ba); !testutils.IsError(err, "end key specified for non-range operation") {
if _, err := Range(ba.Requests); !testutils.IsError(err, "end key specified for non-range operation") {
t.Errorf("unexpected error %v", err)
}
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -703,7 +703,7 @@ func (ds *DistSender) Send(
// Local addressing has already been resolved.
// TODO(tschottdorf): consider rudimentary validation of the batch here
// (for example, non-range requests with EndKey, or empty key ranges).
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
return nil, roachpb.NewError(err)
}
Expand Down Expand Up @@ -840,7 +840,7 @@ func (ds *DistSender) divideAndSendParallelCommit(
// over multiple ranges, so call into divideAndSendBatchToRanges.
qiBa := ba
qiBa.Requests = swappedReqs[swapIdx+1:]
qiRS, err := keys.Range(qiBa)
qiRS, err := keys.Range(qiBa.Requests)
if err != nil {
return br, roachpb.NewError(err)
}
Expand Down Expand Up @@ -882,7 +882,7 @@ func (ds *DistSender) divideAndSendParallelCommit(
// QueryIntent requests. Make sure to determine the request's
// new key span.
ba.Requests = swappedReqs[:swapIdx+1]
rs, err = keys.Range(ba)
rs, err = keys.Range(ba.Requests)
if err != nil {
return nil, roachpb.NewError(err)
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/dist_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1114,7 +1114,7 @@ func TestRetryOnWrongReplicaError(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -1211,7 +1211,7 @@ func TestRetryOnWrongReplicaErrorWithSuggestion(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -1652,7 +1652,7 @@ func TestMultiRangeMergeStaleDescriptor(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -1867,7 +1867,7 @@ func TestTruncateWithSpanAndDescriptor(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -3264,7 +3264,7 @@ func TestEvictMetaRange(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -3431,7 +3431,7 @@ func TestEvictionTokenCoalesce(t *testing.T) {
_ ReplicaSlice,
ba roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
br := ba.CreateReply()
if err != nil {
br.Error = roachpb.NewError(err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -1220,7 +1220,7 @@ func (r *Replica) executeAdminBatch(
return nil, roachpb.NewErrorf("only single-element admin batches allowed")
}

rSpan, err := keys.Range(ba)
rSpan, err := keys.Range(ba.Requests)
if err != nil {
return nil, roachpb.NewError(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ func (r *Replica) evalAndPropose(
}
r.mu.RUnlock()

rSpan, err := keys.Range(ba)
rSpan, err := keys.Range(ba.Requests)
if err != nil {
return nil, nil, 0, roachpb.NewError(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/replica_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func (r *Replica) executeReadOnlyBatch(
return nil, roachpb.NewError(err)
}

rSpan, err := keys.Range(ba)
rSpan, err := keys.Range(ba.Requests)
if err != nil {
return nil, roachpb.NewError(err)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/store_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func (s *Store) TestSender() client.Sender {
// If the client hasn't set ba.Range, we do it a favor and figure out the
// range to which the request needs to go.
//
// NOTE: We don't use keys.Range(ba) here because that does some
// NOTE: We don't use keys.Range(ba.Requests) here because that does some
// validation on the batch, and some tests using this sender don't like
// that.
key, err := keys.Addr(ba.Requests[0].GetInner().Header().Key)
Expand Down Expand Up @@ -150,7 +150,7 @@ func (db *testSender) Send(
return nil, roachpb.NewErrorf("%s method not supported", et.Method())
}
// Lookup range and direct request.
rs, err := keys.Range(ba)
rs, err := keys.Range(ba.Requests)
if err != nil {
return nil, roachpb.NewError(err)
}
Expand Down

0 comments on commit a18b131

Please sign in to comment.