Skip to content

schedule: add region merge checker#839

Merged
Connor1996 merged 69 commits intotikv:masterfrom
Connor1996:region-merge-checker
Mar 22, 2018
Merged

schedule: add region merge checker#839
Connor1996 merged 69 commits intotikv:masterfrom
Connor1996:region-merge-checker

Conversation

@Connor1996
Copy link
Member

in some case, region size is so small even empty, it would be better to merge it with adjacent region if allowed. So introduce region merge checker to detect whether it needs to merge every time receive heartbeat from region. If yes, pd schedule region's peers to make sure that two regions to be merged in same stores, and then launch a merge request to tikv.

@Connor1996 Connor1996 force-pushed the region-merge-checker branch from 5825464 to 16a9ca7 Compare November 9, 2017 07:58
@Connor1996 Connor1996 changed the title [WIP] schedule: add region merge checker [DNM] schedule: add region merge checker Nov 9, 2017
@Connor1996 Connor1996 force-pushed the region-merge-checker branch from aedf20e to b0331ef Compare November 9, 2017 09:17
}

if op1, op2 := c.mergeChecker.Check(region); op1 != nil && op2 != nil {
if c.addOperator(op1) == true {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if c.addOperator(op1) {


if op1, op2 := c.mergeChecker.Check(region); op1 != nil && op2 != nil {
if c.addOperator(op1) == true {
if c.addOperator(op2) == false {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if !c.addOperator(op2) {

if op1, op2 := c.mergeChecker.Check(region); op1 != nil && op2 != nil {
if c.addOperator(op1) == true {
if c.addOperator(op2) == false {
c.removeOperator(op1)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The operator may have been sent to tikv already. I think it will better to check operators before add.

}
c.hbStreams.sendMsg(region, cmd)
case schedule.MergeRegion:
if s.IsFake == true {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if s.IsFake {

var target *core.RegionInfo
prev, next := m.cluster.GetAdjacentRegions(region)
// check key to avoid key range hole
if prev != nil && bytes.Compare(prev.Region.EndKey, region.Region.StartKey) != 0 {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe check it inside getAdjacentRegions?

}

func (m *MergeChecker) matchPeers(source *core.RegionInfo, target *core.RegionInfo) ([]OperatorStep, error) {
storeIDs := make(map[uint64]bool)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We usually use struct{} instead of bool when using map as set.

// IsFinish checks if cuurent step is finished
func (mr MergeRegion) IsFinish(region *core.RegionInfo) bool {
// In view of pd doesn't know merge is success or not
// this step will not finish, so just let operator timeout and then be removed
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about save its key range then check if key range expands?


// Influence calculates the store difference that current step make
func (mr MergeRegion) Influence(opInfluence OpInfluence, region *core.RegionInfo) {
// merge influence nothing
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems it decreases region count, although we don't use it for balancing now.

FromRegion uint64
ToRegion uint64
Direction pdpb.MergeDirection
IsFake bool
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's this used for?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there are two regions involved in merge process, so to keep them from other scheduler, both of them should add operator. But actually, tikv just need one region to get the merge request, thus use a IsFake mark to indicate that this region doesn't need to send merge request to tikv

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think IsFake is reasonable name here, maybe IsPassive. Also need some comments here.

MaxPendingPeerCount uint64 `toml:"max-pending-peer-count,omitempty" json:"max-pending-peer-count"`
// If the size of region is smaller than this value,
// it will try to merge with adjacent regions.
MaxMergeRegionSize uint64 `toml:"max-merge-region-size,omitempty" json:"max-merge-region-size"`
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need another configuration to restrict maximum region size after merge, or the new region may need to split again soon after.

@disksing disksing added the DNM label Nov 10, 2017
@disksing disksing added this to the 1.1 milestone Nov 10, 2017
@Connor1996 Connor1996 force-pushed the region-merge-checker branch 2 times, most recently from ade7b64 to 97fe966 Compare November 10, 2017 05:05
next = i.(*regionItem)
return false
})
item = &regionItem{region: &metapb.Region{StartKey: region.StartKey}}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why assign the item again?

return false
})
item = &regionItem{region: &metapb.Region{StartKey: region.StartKey}}
t.tree.AscendGreaterOrEqual(item, func(i btree.Item) bool {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The AscendGreaterOrEqual will iterate in [item, last], so I am confused that why you set prev here?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the item store in region_tree is in reversed order, namely startkey "a" is greater than "b". https://github.com/Connor1996/pd/blob/e81d31ff04bdb282b66521f09f1562088e8869e7/server/core/region_tree.go#L30

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please add a comment

}
c.hbStreams.sendMsg(region, cmd)
case schedule.MergeRegion:
if s.IsPassive {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what is passive?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you can see the explanation at where it defines

@Connor1996 Connor1996 force-pushed the region-merge-checker branch from eb9e6f5 to 74f28db Compare March 20, 2018 06:13
@siddontang
Copy link
Contributor

do we disable this by default?

func NewRegionWithSiblingCommand() *cobra.Command {
r := &cobra.Command{
Use: "sibling <region_id>",
Short: "show the sibling region of specific region",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sibling regions?


if bytes.Compare(region.StartKey, target.EndKey) != 0 && bytes.Compare(region.EndKey, target.StartKey) != 0 {
return ErrRegionNotAdjacent
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if merge the first region (start key is nil) with the last region (end key is nil) ?

if m.cluster.IsRegionHot(region.GetId()) {
checkerCounter.WithLabelValues("merge_checker", "hot_region").Inc()
return nil, nil
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can also skip the region if its replica count is different from default configuration.

@disksing
Copy link
Contributor

PTAL @nolouch

prev, next := m.cluster.GetAdjacentRegions(region)

target = m.checkTarget(region, prev, target)
target = m.checkTarget(region, next, target)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we also need to consider the state of target peers.

if bytes.Compare(region.StartKey, target.EndKey) != 0 && bytes.Compare(region.EndKey, target.StartKey) != 0 {
// for the case first region (start key is nil) with the last region (end key is nil) but not adjacent
if (bytes.Compare(region.StartKey, target.EndKey) != 0 || region.StartKey == nil) &&
(bytes.Compare(region.EndKey, target.StartKey) != 0 || region.EndKey == nil) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do not use == nil to check empty []byte. Use len(b) == 0 instead.

@disksing
Copy link
Contributor

LGTM.

@Connor1996 Connor1996 force-pushed the region-merge-checker branch from 14a1cc2 to 3ed34eb Compare March 22, 2018 06:32
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants