Skip to content

Commit

Permalink
Use the term router instead of region
Browse files Browse the repository at this point in the history
Signed-off-by: JmPotato <[email protected]>
  • Loading branch information
JmPotato committed Dec 9, 2024
1 parent 13a73e8 commit 15396ee
Show file tree
Hide file tree
Showing 4 changed files with 48 additions and 49 deletions.
45 changes: 28 additions & 17 deletions client/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
"github.com/pingcap/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/tikv/pd/client/clients/metastorage"
"github.com/tikv/pd/client/clients/region"
"github.com/tikv/pd/client/clients/router"
"github.com/tikv/pd/client/clients/tso"
"github.com/tikv/pd/client/constants"
"github.com/tikv/pd/client/errs"
Expand Down Expand Up @@ -71,6 +71,17 @@ type RPCClient interface {
// determine the safepoint for multiple services, it does not trigger a GC
// job. Use UpdateGCSafePoint to trigger the GC job if needed.
UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error)
// ScatterRegion scatters the specified region. Should use it for a batch of regions,
// and the distribution of these regions will be dispersed.
// NOTICE: This method is the old version of ScatterRegions, you should use the later one as your first choice.
ScatterRegion(ctx context.Context, regionID uint64) error
// ScatterRegions scatters the specified regions. Should use it for a batch of regions,
// and the distribution of these regions will be dispersed.
ScatterRegions(ctx context.Context, regionsID []uint64, opts ...opt.RegionsOption) (*pdpb.ScatterRegionResponse, error)
// SplitRegions split regions by given split keys
SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...opt.RegionsOption) (*pdpb.SplitRegionsResponse, error)
// SplitAndScatterRegions split regions by given split keys and scatter new regions
SplitAndScatterRegions(ctx context.Context, splitKeys [][]byte, opts ...opt.RegionsOption) (*pdpb.SplitAndScatterRegionsResponse, error)
// GetOperator gets the status of operator of the specified region.
GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error)

Expand All @@ -96,7 +107,7 @@ type RPCClient interface {
// on your needs.
WithCallerComponent(callerComponent caller.Component) RPCClient

region.Client
router.Client
tso.Client
metastorage.Client
// KeyspaceClient manages keyspace metadata.
Expand Down Expand Up @@ -558,12 +569,12 @@ func (c *client) GetMinTS(ctx context.Context) (physical int64, logical int64, e
return minTS.Physical, minTS.Logical, nil
}

func handleRegionResponse(res *pdpb.GetRegionResponse) *region.Region {
func handleRegionResponse(res *pdpb.GetRegionResponse) *router.Region {
if res.Region == nil {
return nil
}

r := &region.Region{
r := &router.Region{
Meta: res.Region,
Leader: res.Leader,
PendingPeers: res.PendingPeers,
Expand All @@ -576,7 +587,7 @@ func handleRegionResponse(res *pdpb.GetRegionResponse) *region.Region {
}

// GetRegionFromMember implements the RPCClient interface.
func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string, _ ...opt.GetRegionOption) (*region.Region, error) {
func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string, _ ...opt.GetRegionOption) (*router.Region, error) {

Check warning on line 590 in client/client.go

View check run for this annotation

Codecov / codecov/patch

client/client.go#L590

Added line #L590 was not covered by tests
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.GetRegionFromMember", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -615,7 +626,7 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs
}

// GetRegion implements the RPCClient interface.
func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegionOption) (*region.Region, error) {
func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegionOption) (*router.Region, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.GetRegion", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -655,7 +666,7 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio
}

// GetPrevRegion implements the RPCClient interface.
func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetRegionOption) (*region.Region, error) {
func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetRegionOption) (*router.Region, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.GetPrevRegion", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -695,7 +706,7 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetR
}

// GetRegionByID implements the RPCClient interface.
func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt.GetRegionOption) (*region.Region, error) {
func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt.GetRegionOption) (*router.Region, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.GetRegionByID", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -735,7 +746,7 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt
}

// ScanRegions implements the RPCClient interface.
func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int, opts ...opt.GetRegionOption) ([]*region.Region, error) {
func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int, opts ...opt.GetRegionOption) ([]*router.Region, error) {

Check warning on line 749 in client/client.go

View check run for this annotation

Codecov / codecov/patch

client/client.go#L749

Added line #L749 was not covered by tests
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.ScanRegions", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -786,7 +797,7 @@ func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int,
}

// BatchScanRegions implements the RPCClient interface.
func (c *client) BatchScanRegions(ctx context.Context, ranges []region.KeyRange, limit int, opts ...opt.GetRegionOption) ([]*region.Region, error) {
func (c *client) BatchScanRegions(ctx context.Context, ranges []router.KeyRange, limit int, opts ...opt.GetRegionOption) ([]*router.Region, error) {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span = span.Tracer().StartSpan("pdclient.BatchScanRegions", opentracing.ChildOf(span.Context()))
defer span.Finish()
Expand Down Expand Up @@ -839,10 +850,10 @@ func (c *client) BatchScanRegions(ctx context.Context, ranges []region.KeyRange,
return handleBatchRegionsResponse(resp), nil
}

func handleBatchRegionsResponse(resp *pdpb.BatchScanRegionsResponse) []*region.Region {
regions := make([]*region.Region, 0, len(resp.GetRegions()))
func handleBatchRegionsResponse(resp *pdpb.BatchScanRegionsResponse) []*router.Region {
regions := make([]*router.Region, 0, len(resp.GetRegions()))
for _, r := range resp.GetRegions() {
region := &region.Region{
region := &router.Region{
Meta: r.Region,
Leader: r.Leader,
PendingPeers: r.PendingPeers,
Expand All @@ -856,21 +867,21 @@ func handleBatchRegionsResponse(resp *pdpb.BatchScanRegionsResponse) []*region.R
return regions
}

func handleRegionsResponse(resp *pdpb.ScanRegionsResponse) []*region.Region {
var regions []*region.Region
func handleRegionsResponse(resp *pdpb.ScanRegionsResponse) []*router.Region {
var regions []*router.Region

Check warning on line 871 in client/client.go

View check run for this annotation

Codecov / codecov/patch

client/client.go#L870-L871

Added lines #L870 - L871 were not covered by tests
if len(resp.GetRegions()) == 0 {
// Make it compatible with old server.
metas, leaders := resp.GetRegionMetas(), resp.GetLeaders()
for i := range metas {
r := &region.Region{Meta: metas[i]}
r := &router.Region{Meta: metas[i]}

Check warning on line 876 in client/client.go

View check run for this annotation

Codecov / codecov/patch

client/client.go#L876

Added line #L876 was not covered by tests
if i < len(leaders) {
r.Leader = leaders[i]
}
regions = append(regions, r)
}
} else {
for _, r := range resp.GetRegions() {
region := &region.Region{
region := &router.Region{

Check warning on line 884 in client/client.go

View check run for this annotation

Codecov / codecov/patch

client/client.go#L884

Added line #L884 was not covered by tests
Meta: r.Region,
Leader: r.Leader,
PendingPeers: r.PendingPeers,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,15 +12,14 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package region
package router

import (
"context"
"encoding/hex"
"net/url"

"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/pdpb"
"github.com/tikv/pd/client/opt"
)

Expand Down Expand Up @@ -65,7 +64,7 @@ func (r *KeyRange) EscapeAsHexStr() (startKeyStr, endKeyStr string) {
return
}

// Client defines the interface of a region client.
// Client defines the interface of a router client, which includes the methods for obtaining the routing information.
type Client interface {
// GetRegion gets a region and its leader Peer from PD by key.
// The region may expire after split. Caller is responsible for caching and
Expand All @@ -91,15 +90,4 @@ type Client interface {
// with empty value (PeerID is 0).
// The returned regions are flattened, even there are key ranges located in the same region, only one region will be returned.
BatchScanRegions(ctx context.Context, keyRanges []KeyRange, limit int, opts ...opt.GetRegionOption) ([]*Region, error)
// ScatterRegion scatters the specified region. Should use it for a batch of regions,
// and the distribution of these regions will be dispersed.
// NOTICE: This method is the old version of ScatterRegions, you should use the later one as your first choice.
ScatterRegion(ctx context.Context, regionID uint64) error
// ScatterRegions scatters the specified regions. Should use it for a batch of regions,
// and the distribution of these regions will be dispersed.
ScatterRegions(ctx context.Context, regionsID []uint64, opts ...opt.RegionsOption) (*pdpb.ScatterRegionResponse, error)
// SplitRegions split regions by given split keys
SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...opt.RegionsOption) (*pdpb.SplitRegionsResponse, error)
// SplitAndScatterRegions split regions by given split keys and scatter new regions
SplitAndScatterRegions(ctx context.Context, splitKeys [][]byte, opts ...opt.RegionsOption) (*pdpb.SplitAndScatterRegionsResponse, error)
}
2 changes: 1 addition & 1 deletion client/http/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
"github.com/pingcap/kvproto/pkg/encryptionpb"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/kvproto/pkg/pdpb"
pd "github.com/tikv/pd/client/clients/region"
pd "github.com/tikv/pd/client/clients/router"
)

// ServiceSafePoint is the safepoint for a specific service
Expand Down
34 changes: 17 additions & 17 deletions tests/integrations/client/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
pd "github.com/tikv/pd/client"
"github.com/tikv/pd/client/clients/region"
"github.com/tikv/pd/client/clients/router"
"github.com/tikv/pd/client/opt"
"github.com/tikv/pd/client/pkg/caller"
"github.com/tikv/pd/client/pkg/retry"
Expand Down Expand Up @@ -540,11 +540,11 @@ func (suite *followerForwardAndHandleTestSuite) TestGetTsoByFollowerForwarding1(
checkTS(re, cli, lastTS)

re.NoError(failpoint.Enable("github.com/tikv/pd/client/responseNil", "return(true)"))
regions, err := cli.BatchScanRegions(ctx, []region.KeyRange{{StartKey: []byte(""), EndKey: []byte("")}}, 100)
regions, err := cli.BatchScanRegions(ctx, []router.KeyRange{{StartKey: []byte(""), EndKey: []byte("")}}, 100)
re.NoError(err)
re.Empty(regions)
re.NoError(failpoint.Disable("github.com/tikv/pd/client/responseNil"))
regions, err = cli.BatchScanRegions(ctx, []region.KeyRange{{StartKey: []byte(""), EndKey: []byte("")}}, 100)
regions, err = cli.BatchScanRegions(ctx, []router.KeyRange{{StartKey: []byte(""), EndKey: []byte("")}}, 100)
re.NoError(err)
re.Len(regions, 1)
}
Expand Down Expand Up @@ -1217,7 +1217,7 @@ func (suite *clientTestSuite) TestScanRegions() {

// Wait for region heartbeats.
testutil.Eventually(re, func() bool {
scanRegions, err := suite.client.BatchScanRegions(context.Background(), []region.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10)
scanRegions, err := suite.client.BatchScanRegions(context.Background(), []router.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10)
return err == nil && len(scanRegions) == 10
})

Expand All @@ -1235,7 +1235,7 @@ func (suite *clientTestSuite) TestScanRegions() {

t := suite.T()
check := func(start, end []byte, limit int, expect []*metapb.Region) {
scanRegions, err := suite.client.BatchScanRegions(context.Background(), []region.KeyRange{{StartKey: start, EndKey: end}}, limit)
scanRegions, err := suite.client.BatchScanRegions(context.Background(), []router.KeyRange{{StartKey: start, EndKey: end}}, limit)
re.NoError(err)
re.Len(scanRegions, len(expect))
t.Log("scanRegions", scanRegions)
Expand Down Expand Up @@ -1850,7 +1850,7 @@ func (suite *clientTestSuite) TestBatchScanRegions() {

// Wait for region heartbeats.
testutil.Eventually(re, func() bool {
scanRegions, err := suite.client.BatchScanRegions(ctx, []region.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10)
scanRegions, err := suite.client.BatchScanRegions(ctx, []router.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10)
return err == nil && len(scanRegions) == 10
})

Expand All @@ -1872,7 +1872,7 @@ func (suite *clientTestSuite) TestBatchScanRegions() {

t := suite.T()
var outputMustContainAllKeyRangeOptions []bool
check := func(ranges []region.KeyRange, limit int, expect []*metapb.Region) {
check := func(ranges []router.KeyRange, limit int, expect []*metapb.Region) {
for _, bucket := range []bool{false, true} {
for _, outputMustContainAllKeyRange := range outputMustContainAllKeyRangeOptions {
var opts []opt.GetRegionOption
Expand Down Expand Up @@ -1918,16 +1918,16 @@ func (suite *clientTestSuite) TestBatchScanRegions() {

// valid ranges
outputMustContainAllKeyRangeOptions = []bool{false, true}
check([]region.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10, regions)
check([]region.KeyRange{{StartKey: []byte{1}, EndKey: nil}}, 5, regions[1:6])
check([]region.KeyRange{
check([]router.KeyRange{{StartKey: []byte{0}, EndKey: nil}}, 10, regions)
check([]router.KeyRange{{StartKey: []byte{1}, EndKey: nil}}, 5, regions[1:6])
check([]router.KeyRange{
{StartKey: []byte{0}, EndKey: []byte{1}},
{StartKey: []byte{2}, EndKey: []byte{3}},
{StartKey: []byte{4}, EndKey: []byte{5}},
{StartKey: []byte{6}, EndKey: []byte{7}},
{StartKey: []byte{8}, EndKey: []byte{9}},
}, 10, []*metapb.Region{regions[0], regions[2], regions[4], regions[6], regions[8]})
check([]region.KeyRange{
check([]router.KeyRange{
{StartKey: []byte{0}, EndKey: []byte{1}},
{StartKey: []byte{2}, EndKey: []byte{3}},
{StartKey: []byte{4}, EndKey: []byte{5}},
Expand All @@ -1936,7 +1936,7 @@ func (suite *clientTestSuite) TestBatchScanRegions() {
}, 3, []*metapb.Region{regions[0], regions[2], regions[4]})

outputMustContainAllKeyRangeOptions = []bool{false}
check([]region.KeyRange{
check([]router.KeyRange{
{StartKey: []byte{0}, EndKey: []byte{0, 1}}, // non-continuous ranges in a region
{StartKey: []byte{0, 2}, EndKey: []byte{0, 3}},
{StartKey: []byte{0, 3}, EndKey: []byte{0, 4}},
Expand All @@ -1945,26 +1945,26 @@ func (suite *clientTestSuite) TestBatchScanRegions() {
{StartKey: []byte{4}, EndKey: []byte{5}},
}, 10, []*metapb.Region{regions[0], regions[1], regions[2], regions[4]})
outputMustContainAllKeyRangeOptions = []bool{false}
check([]region.KeyRange{
check([]router.KeyRange{
{StartKey: []byte{9}, EndKey: []byte{10, 1}},
}, 10, []*metapb.Region{regions[9]})

// invalid ranges
_, err := suite.client.BatchScanRegions(
ctx,
[]region.KeyRange{{StartKey: []byte{1}, EndKey: []byte{0}}},
[]router.KeyRange{{StartKey: []byte{1}, EndKey: []byte{0}}},
10,
opt.WithOutputMustContainAllKeyRange(),
)
re.ErrorContains(err, "invalid key range, start key > end key")
_, err = suite.client.BatchScanRegions(ctx, []region.KeyRange{
_, err = suite.client.BatchScanRegions(ctx, []router.KeyRange{
{StartKey: []byte{0}, EndKey: []byte{2}},
{StartKey: []byte{1}, EndKey: []byte{3}},
}, 10)
re.ErrorContains(err, "invalid key range, ranges overlapped")
_, err = suite.client.BatchScanRegions(
ctx,
[]region.KeyRange{{StartKey: []byte{9}, EndKey: []byte{10, 1}}},
[]router.KeyRange{{StartKey: []byte{9}, EndKey: []byte{10, 1}}},
10,
opt.WithOutputMustContainAllKeyRange(),
)
Expand All @@ -1989,7 +1989,7 @@ func (suite *clientTestSuite) TestBatchScanRegions() {
testutil.Eventually(re, func() bool {
_, err = suite.client.BatchScanRegions(
ctx,
[]region.KeyRange{{StartKey: []byte{9}, EndKey: []byte{101}}},
[]router.KeyRange{{StartKey: []byte{9}, EndKey: []byte{101}}},
10,
opt.WithOutputMustContainAllKeyRange(),
)
Expand Down

0 comments on commit 15396ee

Please sign in to comment.