Skip to content

Commit

Permalink
Merge branch 'master' into issue-28605
Browse files Browse the repository at this point in the history
  • Loading branch information
tisonkun authored Feb 18, 2022
2 parents 1513753 + d20be03 commit bfcf39f
Show file tree
Hide file tree
Showing 25 changed files with 706 additions and 220 deletions.
25 changes: 12 additions & 13 deletions br/pkg/lightning/backend/local/localhelper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,11 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/stretchr/testify/require"
"github.com/tikv/pd/server/core"
"github.com/tikv/pd/server/schedule/placement"
"go.uber.org/atomic"
)

Expand All @@ -51,7 +50,7 @@ type testClient struct {
mu sync.RWMutex
stores map[uint64]*metapb.Store
regions map[uint64]*restore.RegionInfo
regionsInfo *core.RegionsInfo // For now it's only used in ScanRegions
regionsInfo *pdtypes.RegionTree // For now it's only used in ScanRegions
nextRegionID uint64
splitCount atomic.Int32
hook clientHook
Expand All @@ -63,9 +62,9 @@ func newTestClient(
nextRegionID uint64,
hook clientHook,
) *testClient {
regionsInfo := core.NewRegionsInfo()
regionsInfo := &pdtypes.RegionTree{}
for _, regionInfo := range regions {
regionsInfo.SetRegion(core.NewRegionInfo(regionInfo.Region, regionInfo.Leader))
regionsInfo.SetRegion(pdtypes.NewRegionInfo(regionInfo.Region, regionInfo.Leader))
}
return &testClient{
stores: stores,
Expand Down Expand Up @@ -150,12 +149,12 @@ func (c *testClient) SplitRegion(
},
}
c.regions[c.nextRegionID] = newRegion
c.regionsInfo.SetRegion(core.NewRegionInfo(newRegion.Region, newRegion.Leader))
c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(newRegion.Region, newRegion.Leader))
c.nextRegionID++
target.Region.StartKey = splitKey
target.Region.RegionEpoch.ConfVer++
c.regions[target.Region.Id] = target
c.regionsInfo.SetRegion(core.NewRegionInfo(target.Region, target.Leader))
c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(target.Region, target.Leader))
return newRegion, nil
}

Expand Down Expand Up @@ -211,15 +210,15 @@ func (c *testClient) BatchSplitRegionsWithOrigin(
},
}
c.regions[c.nextRegionID] = newRegion
c.regionsInfo.SetRegion(core.NewRegionInfo(newRegion.Region, newRegion.Leader))
c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(newRegion.Region, newRegion.Leader))
c.nextRegionID++
startKey = key
newRegions = append(newRegions, newRegion)
}
if !bytes.Equal(target.Region.StartKey, startKey) {
target.Region.StartKey = startKey
c.regions[target.Region.Id] = target
c.regionsInfo.SetRegion(core.NewRegionInfo(target.Region, target.Leader))
c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(target.Region, target.Leader))
}

if len(newRegions) == 0 {
Expand Down Expand Up @@ -260,8 +259,8 @@ func (c *testClient) ScanRegions(ctx context.Context, key, endKey []byte, limit
regions := make([]*restore.RegionInfo, 0, len(infos))
for _, info := range infos {
regions = append(regions, &restore.RegionInfo{
Region: info.GetMeta(),
Leader: info.GetLeader(),
Region: info.Meta,
Leader: info.Leader,
})
}

Expand All @@ -272,11 +271,11 @@ func (c *testClient) ScanRegions(ctx context.Context, key, endKey []byte, limit
return regions, err
}

func (c *testClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r placement.Rule, err error) {
func (c *testClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r pdtypes.Rule, err error) {
return
}

func (c *testClient) SetPlacementRule(ctx context.Context, rule placement.Rule) error {
func (c *testClient) SetPlacementRule(ctx context.Context, rule pdtypes.Rule) error {
return nil
}

Expand Down
19 changes: 9 additions & 10 deletions br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,10 @@ import (
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
"github.com/tikv/pd/server/api"
pdconfig "github.com/tikv/pd/server/config"
)

const (
Expand All @@ -78,7 +77,7 @@ func (rc *Controller) isSourceInLocal() bool {
}

func (rc *Controller) getReplicaCount(ctx context.Context) (uint64, error) {
result := &pdconfig.ReplicationConfig{}
result := &pdtypes.ReplicationConfig{}
err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdReplicate, &result)
if err != nil {
return 0, errors.Trace(err)
Expand All @@ -87,7 +86,7 @@ func (rc *Controller) getReplicaCount(ctx context.Context) (uint64, error) {
}

func (rc *Controller) getClusterAvail(ctx context.Context) (uint64, error) {
result := &api.StoresInfo{}
result := &pdtypes.StoresInfo{}
if err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, result); err != nil {
return 0, errors.Trace(err)
}
Expand Down Expand Up @@ -183,7 +182,7 @@ func (rc *Controller) ClusterIsAvailable(ctx context.Context) error {
return nil
}

func isTiFlash(store *api.MetaStore) bool {
func isTiFlash(store *pdtypes.MetaStore) bool {
for _, label := range store.Labels {
if label.Key == "engine" && label.Value == "tiflash" {
return true
Expand All @@ -198,7 +197,7 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error {
defer func() {
rc.checkTemplate.Collect(Critical, passed, message)
}()
storeInfo := &api.StoresInfo{}
storeInfo := &pdtypes.StoresInfo{}
err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, storeInfo)
if err != nil {
return errors.Trace(err)
Expand All @@ -207,12 +206,12 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error {
return nil
}

var result api.RegionsInfo
var result pdtypes.RegionsInfo
if err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdEmptyRegions, &result); err != nil {
return errors.Trace(err)
}
regions := make(map[uint64]int)
stores := make(map[uint64]*api.StoreInfo)
stores := make(map[uint64]*pdtypes.StoreInfo)
for _, region := range result.Regions {
for _, peer := range region.Peers {
regions[peer.StoreId]++
Expand Down Expand Up @@ -275,12 +274,12 @@ func (rc *Controller) checkRegionDistribution(ctx context.Context) error {
rc.checkTemplate.Collect(Critical, passed, message)
}()

result := &api.StoresInfo{}
result := &pdtypes.StoresInfo{}
err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, result)
if err != nil {
return errors.Trace(err)
}
stores := make([]*api.StoreInfo, 0, len(result.Stores))
stores := make([]*pdtypes.StoreInfo, 0, len(result.Stores))
for _, store := range result.Stores {
if metapb.StoreState(metapb.StoreState_value[store.Store.StateName]) != metapb.StoreState_Up {
continue
Expand Down
48 changes: 24 additions & 24 deletions br/pkg/lightning/restore/table_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,12 +58,12 @@ import (
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
tmock "github.com/pingcap/tidb/util/mock"
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"
"github.com/tikv/pd/server/api"
)

type tableRestoreSuiteBase struct {
Expand Down Expand Up @@ -1116,41 +1116,41 @@ func (mockTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(ta

func (s *tableRestoreSuite) TestCheckClusterRegion() {
type testCase struct {
stores api.StoresInfo
emptyRegions api.RegionsInfo
stores pdtypes.StoresInfo
emptyRegions pdtypes.RegionsInfo
expectMsgs []string
expectResult bool
expectErrorCnt int
}

makeRegions := func(regionCnt int, storeID uint64) []api.RegionInfo {
var regions []api.RegionInfo
makeRegions := func(regionCnt int, storeID uint64) []pdtypes.RegionInfo {
var regions []pdtypes.RegionInfo
for i := 0; i < regionCnt; i++ {
regions = append(regions, api.RegionInfo{Peers: []api.MetaPeer{{Peer: &metapb.Peer{StoreId: storeID}}}})
regions = append(regions, pdtypes.RegionInfo{Peers: []pdtypes.MetaPeer{{Peer: &metapb.Peer{StoreId: storeID}}}})
}
return regions
}

testCases := []testCase{
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 200}},
stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 200}},
}},
emptyRegions: api.RegionsInfo{
Regions: append([]api.RegionInfo(nil), makeRegions(100, 1)...),
emptyRegions: pdtypes.RegionsInfo{
Regions: append([]pdtypes.RegionInfo(nil), makeRegions(100, 1)...),
},
expectMsgs: []string{".*Cluster doesn't have too many empty regions.*", ".*Cluster region distribution is balanced.*"},
expectResult: true,
expectErrorCnt: 0,
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 2000}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 3100}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 2000}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 3100}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}},
}},
emptyRegions: api.RegionsInfo{
Regions: append(append(append([]api.RegionInfo(nil),
emptyRegions: pdtypes.RegionsInfo{
Regions: append(append(append([]pdtypes.RegionInfo(nil),
makeRegions(600, 1)...),
makeRegions(300, 2)...),
makeRegions(1200, 3)...),
Expand All @@ -1164,20 +1164,20 @@ func (s *tableRestoreSuite) TestCheckClusterRegion() {
expectErrorCnt: 1,
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 1200}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 3000}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 1200}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 3000}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}},
}},
expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"},
expectResult: false,
expectErrorCnt: 1,
},
{
stores: api.StoresInfo{Stores: []*api.StoreInfo{
{Store: &api.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &api.StoreStatus{RegionCount: 0}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &api.StoreStatus{RegionCount: 2800}},
{Store: &api.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &api.StoreStatus{RegionCount: 2500}},
stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 0}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 2800}},
{Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}},
}},
expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"},
expectResult: false,
Expand Down
8 changes: 4 additions & 4 deletions br/pkg/pdutil/pd.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,9 @@ import (
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/httputil"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/util/codec"
pd "github.com/tikv/pd/client"
pdapi "github.com/tikv/pd/server/api"
"go.uber.org/zap"
"google.golang.org/grpc"
)
Expand Down Expand Up @@ -350,12 +350,12 @@ func (p *PdController) getRegionCountWith(
}

// GetStoreInfo returns the info of store with the specified id.
func (p *PdController) GetStoreInfo(ctx context.Context, storeID uint64) (*pdapi.StoreInfo, error) {
func (p *PdController) GetStoreInfo(ctx context.Context, storeID uint64) (*pdtypes.StoreInfo, error) {
return p.getStoreInfoWith(ctx, pdRequest, storeID)
}

func (p *PdController) getStoreInfoWith(
ctx context.Context, get pdHTTPRequest, storeID uint64) (*pdapi.StoreInfo, error) {
ctx context.Context, get pdHTTPRequest, storeID uint64) (*pdtypes.StoreInfo, error) {
var err error
for _, addr := range p.addrs {
query := fmt.Sprintf(
Expand All @@ -366,7 +366,7 @@ func (p *PdController) getStoreInfoWith(
err = e
continue
}
store := pdapi.StoreInfo{}
store := pdtypes.StoreInfo{}
err = json.Unmarshal(v, &store)
if err != nil {
return nil, errors.Trace(err)
Expand Down
27 changes: 12 additions & 15 deletions br/pkg/pdutil/pd_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,9 @@ import (
"github.com/coreos/go-semver/semver"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/store/pdtypes"
"github.com/pingcap/tidb/util/codec"
"github.com/stretchr/testify/require"
"github.com/tikv/pd/pkg/typeutil"
"github.com/tikv/pd/server/api"
"github.com/tikv/pd/server/core"
"github.com/tikv/pd/server/statistics"
)

func TestScheduler(t *testing.T) {
Expand Down Expand Up @@ -106,26 +103,26 @@ func TestGetClusterVersion(t *testing.T) {
}

func TestRegionCount(t *testing.T) {
regions := core.NewRegionsInfo()
regions.SetRegion(core.NewRegionInfo(&metapb.Region{
regions := &pdtypes.RegionTree{}
regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{
Id: 1,
StartKey: codec.EncodeBytes(nil, []byte{1, 1}),
EndKey: codec.EncodeBytes(nil, []byte{1, 3}),
RegionEpoch: &metapb.RegionEpoch{},
}, nil))
regions.SetRegion(core.NewRegionInfo(&metapb.Region{
regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{
Id: 2,
StartKey: codec.EncodeBytes(nil, []byte{1, 3}),
EndKey: codec.EncodeBytes(nil, []byte{1, 5}),
RegionEpoch: &metapb.RegionEpoch{},
}, nil))
regions.SetRegion(core.NewRegionInfo(&metapb.Region{
regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{
Id: 3,
StartKey: codec.EncodeBytes(nil, []byte{2, 3}),
EndKey: codec.EncodeBytes(nil, []byte{3, 4}),
RegionEpoch: &metapb.RegionEpoch{},
}, nil))
require.Equal(t, 3, regions.Len())
require.Equal(t, 3, len(regions.Regions))

mock := func(
_ context.Context, addr string, prefix string, _ *http.Client, _ string, _ io.Reader,
Expand All @@ -138,7 +135,7 @@ func TestRegionCount(t *testing.T) {
t.Log(hex.EncodeToString([]byte(start)))
t.Log(hex.EncodeToString([]byte(end)))
scanRegions := regions.ScanRange([]byte(start), []byte(end), 0)
stats := statistics.RegionStats{Count: len(scanRegions)}
stats := pdtypes.RegionStats{Count: len(scanRegions)}
ret, err := json.Marshal(stats)
require.NoError(t, err)
return ret, nil
Expand Down Expand Up @@ -206,12 +203,12 @@ func TestPDRequestRetry(t *testing.T) {
}

func TestStoreInfo(t *testing.T) {
storeInfo := api.StoreInfo{
Status: &api.StoreStatus{
Capacity: typeutil.ByteSize(1024),
Available: typeutil.ByteSize(1024),
storeInfo := pdtypes.StoreInfo{
Status: &pdtypes.StoreStatus{
Capacity: pdtypes.ByteSize(1024),
Available: pdtypes.ByteSize(1024),
},
Store: &api.MetaStore{
Store: &pdtypes.MetaStore{
StateName: "Tombstone",
},
}
Expand Down
Loading

0 comments on commit bfcf39f

Please sign in to comment.