Skip to content

Commit

Permalink
tools/simulator: replace GenerateSplitKey with GenerateTableKeys (#8332)
Browse files Browse the repository at this point in the history
ref #8135

Signed-off-by: husharp <[email protected]>
  • Loading branch information
HuSharp authored Jun 27, 2024
1 parent 1578f29 commit ab7f903
Show file tree
Hide file tree
Showing 9 changed files with 91 additions and 138 deletions.
2 changes: 1 addition & 1 deletion tests/integrations/realcluster/scheduler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func TestRegionLabelDenyScheduler(t *testing.T) {

regions, err := pdHTTPCli.GetRegions(ctx)
re.NoError(err)
re.GreaterOrEqual(len(regions.Regions), 1)
re.NotEmpty(regions.Regions)
region1 := regions.Regions[0]

err = pdHTTPCli.DeleteScheduler(ctx, schedulers.BalanceLeaderName)
Expand Down
35 changes: 23 additions & 12 deletions tools/pd-simulator/simulator/cases/region_split.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/tikv/pd/pkg/core"
sc "github.com/tikv/pd/tools/pd-simulator/simulator/config"
"github.com/tikv/pd/tools/pd-simulator/simulator/info"
"github.com/tikv/pd/tools/pd-simulator/simulator/simutil"
)

func newRegionSplit(config *sc.SimConfig) *Case {
Expand All @@ -28,23 +29,33 @@ func newRegionSplit(config *sc.SimConfig) *Case {
allStores := make(map[uint64]struct{}, totalStore)

for i := 0; i < totalStore; i++ {
id := uint64(i)
storeID := simutil.IDAllocator.NextID()
simCase.Stores = append(simCase.Stores, &Store{
ID: id,
ID: storeID,
Status: metapb.StoreState_Up,
})
allStores[id] = struct{}{}
allStores[storeID] = struct{}{}
}
peers := []*metapb.Peer{
{Id: 4, StoreId: 1},
replica := int(config.ServerConfig.Replication.MaxReplicas)
peers := make([]*metapb.Peer, 0, replica)
for j := 0; j < replica; j++ {
peers = append(peers, &metapb.Peer{
Id: simutil.IDAllocator.NextID(),
StoreId: uint64((j)%(totalStore-1) + 1),
})
}
regionID := simutil.IDAllocator.NextID()
simCase.Regions = []Region{
{
ID: regionID,
Peers: peers,
Leader: peers[0],
Size: 1 * units.MiB,
Keys: 10000,
},
}
simCase.Regions = append(simCase.Regions, Region{
ID: 5,
Peers: peers,
Leader: peers[0],
Size: 1 * units.MiB,
Keys: 10000,
})
// update total region
config.TotalRegion = 1

simCase.RegionSplitSize = 128 * units.MiB
simCase.RegionSplitKeys = 10000
Expand Down
5 changes: 5 additions & 0 deletions tools/pd-simulator/simulator/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,6 +206,10 @@ func (c *client) reportRegionHeartbeat(ctx context.Context, stream pdpb.PD_Regio
for {
select {
case r := <-c.reportRegionHeartbeatCh:
if r == nil {
simutil.Logger.Error("report nil regionHeartbeat error",
zap.String("tag", c.tag), zap.Error(errors.New("nil region")))
}
region := r.Clone()
request := &pdpb.RegionHeartbeatRequest{
Header: requestHeader(),
Expand Down Expand Up @@ -539,6 +543,7 @@ func PutPDConfig(config *sc.PDConfig) error {
}

func ChooseToHaltPDSchedule(halt bool) {
HaltSchedule = halt
PDHTTPClient.SetConfig(context.Background(), map[string]any{
"schedule.halt-scheduling": strconv.FormatBool(halt),
})
Expand Down
5 changes: 5 additions & 0 deletions tools/pd-simulator/simulator/drive.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,8 +176,13 @@ func (d *Driver) Tick() {
d.wg.Wait()
}

var HaltSchedule = false

// Check checks if the simulation is completed.
func (d *Driver) Check() bool {
if !HaltSchedule {
return false
}
length := uint64(len(d.conn.Nodes) + 1)
var stores []*metapb.Store
for index, s := range d.conn.Nodes {
Expand Down
2 changes: 1 addition & 1 deletion tools/pd-simulator/simulator/event.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ func (e *WriteFlowOnSpot) Run(raft *RaftEngine, tickCount int64) bool {
region := raft.GetRegionByKey([]byte(key))
simutil.Logger.Debug("search the region", zap.Reflect("region", region.GetMeta()))
if region == nil {
simutil.Logger.Error("region not found for key", zap.String("key", key))
simutil.Logger.Error("region not found for key", zap.String("key", key), zap.Any("byte(key)", []byte(key)))
continue
}
raft.updateRegionStore(region, size)
Expand Down
7 changes: 7 additions & 0 deletions tools/pd-simulator/simulator/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,9 @@ func (n *Node) regionHeartBeat() {
n.raftEngine.TraverseRegions(func(region *core.RegionInfo) {
if region.GetLeader() != nil && region.GetLeader().GetStoreId() == n.Id {
ctx, cancel := context.WithTimeout(n.ctx, pdTimeout)
if region == nil {
simutil.Logger.Fatal("region not found")
}
err := n.client.RegionHeartbeat(ctx, region)
if err != nil {
simutil.Logger.Info("report region heartbeat error",
Expand All @@ -225,6 +228,10 @@ func (n *Node) reportRegionChange() {
for _, regionID := range regionIDs {
region := n.raftEngine.GetRegion(regionID)
ctx, cancel := context.WithTimeout(n.ctx, pdTimeout)
if region == nil {
simutil.Logger.Info("region not found",
zap.Uint64("region-id", regionID), zap.Uint64("node-id", n.Id))
}
err := n.client.RegionHeartbeat(ctx, region)
if err != nil {
simutil.Logger.Info("report region change heartbeat error",
Expand Down
34 changes: 11 additions & 23 deletions tools/pd-simulator/simulator/raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,12 @@ import (
// RaftEngine records all raft information.
type RaftEngine struct {
syncutil.RWMutex
regionsInfo *core.RegionsInfo
conn *Connection
regionChange map[uint64][]uint64
regionSplitSize int64
regionSplitKeys int64
storeConfig *config.SimConfig
useTiDBEncodedKey bool
regionsInfo *core.RegionsInfo
conn *Connection
regionChange map[uint64][]uint64
regionSplitSize int64
regionSplitKeys int64
storeConfig *config.SimConfig
}

// NewRaftEngine creates the initialized raft with the configuration.
Expand All @@ -49,14 +48,7 @@ func NewRaftEngine(conf *cases.Case, conn *Connection, storeConfig *config.SimCo
regionSplitKeys: conf.RegionSplitKeys,
storeConfig: storeConfig,
}
var splitKeys []string
if conf.TableNumber > 0 {
splitKeys = simutil.GenerateTableKeys(conf.TableNumber, len(conf.Regions)-1)
r.useTiDBEncodedKey = true
} else {
splitKeys = simutil.GenerateKeys(len(conf.Regions) - 1)
}

splitKeys := simutil.GenerateTableKeys(conf.TableNumber, len(conf.Regions)-1)
for i, region := range conf.Regions {
meta := &metapb.Region{
Id: region.ID,
Expand Down Expand Up @@ -133,14 +125,9 @@ func (r *RaftEngine) stepSplit(region *core.RegionInfo) {
}
}

var splitKey []byte
if r.useTiDBEncodedKey {
splitKey, err = simutil.GenerateTiDBEncodedSplitKey(region.GetStartKey(), region.GetEndKey())
if err != nil {
simutil.Logger.Fatal("Generate TiDB encoded split key failed", zap.Error(err))
}
} else {
splitKey = simutil.GenerateSplitKey(region.GetStartKey(), region.GetEndKey())
splitKey, err := simutil.GenerateTiDBEncodedSplitKey(region.GetStartKey(), region.GetEndKey())
if err != nil {
simutil.Logger.Fatal("Generate TiDB encoded split key failed", zap.Error(err))
}
left := region.Clone(
core.WithNewRegionID(ids[len(ids)-1]),
Expand All @@ -162,6 +149,7 @@ func (r *RaftEngine) stepSplit(region *core.RegionInfo) {
r.SetRegion(right)
r.SetRegion(left)
simutil.Logger.Debug("region split",
zap.Uint64("node-id", region.GetLeader().GetStoreId()),
zap.Uint64("region-id", region.GetID()),
zap.Reflect("origin", region.GetMeta()),
zap.Reflect("left", left.GetMeta()),
Expand Down
55 changes: 4 additions & 51 deletions tools/pd-simulator/simulator/simutil/key.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,38 +16,11 @@ package simutil

import (
"bytes"
"math/rand"
"sort"

"github.com/pingcap/errors"
"github.com/tikv/pd/pkg/codec"
)

const (
// 26^10 ~= 1.4e+14, should be enough.
keyChars = "abcdefghijklmnopqrstuvwxyz"
keyLen = 10
)

// GenerateKeys generates ordered, unique strings.
func GenerateKeys(size int) []string {
m := make(map[string]struct{}, size)
for len(m) < size {
k := make([]byte, keyLen)
for i := range k {
k[i] = keyChars[rand.Intn(len(keyChars))]
}
m[string(k)] = struct{}{}
}

v := make([]string, 0, size)
for k := range m {
v = append(v, k)
}
sort.Strings(v)
return v
}

// GenerateTableKey generates the table key according to the table ID and row ID.
func GenerateTableKey(tableID, rowID int64) []byte {
key := codec.GenerateRowKey(tableID, rowID)
Expand All @@ -59,6 +32,10 @@ func GenerateTableKey(tableID, rowID int64) []byte {

// GenerateTableKeys generates the table keys according to the table count and size.
func GenerateTableKeys(tableCount, size int) []string {
if tableCount <= 0 {
// set default tableCount as 1
tableCount = 1
}
v := make([]string, 0, size)
groupNumber := size / tableCount
tableID := 0
Expand All @@ -74,30 +51,6 @@ func GenerateTableKeys(tableCount, size int) []string {
return v
}

// GenerateSplitKey generate the split key.
func GenerateSplitKey(start, end []byte) []byte {
key := make([]byte, 0, len(start))
// lessThanEnd is set as true when the key is already less than end key.
lessThanEnd := len(end) == 0
for i, s := range start {
e := byte('z')
if !lessThanEnd {
e = end[i]
}
c := (s + e) / 2
key = append(key, c)
// case1: s = c < e. Continue with lessThanEnd=true.
// case2: s < c < e. return key.
// case3: s = c = e. Continue with lessThanEnd=false.
lessThanEnd = c < e
if c > s && c < e {
return key
}
}
key = append(key, ('a'+'z')/2)
return key
}

func mustDecodeMvccKey(key []byte) ([]byte, error) {
// FIXME: seems nil key not encode to order compare key
if len(key) == 0 {
Expand Down
84 changes: 34 additions & 50 deletions tools/pd-simulator/simulator/simutil/key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,8 @@
package simutil

import (
"strings"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/tikv/pd/pkg/core"
"testing"

"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -97,58 +98,41 @@ func TestGenerateTiDBEncodedSplitKey(t *testing.T) {
re.Error(err)
}

func TestGenerateKeys(t *testing.T) {
func TestRegionSplitKey(t *testing.T) {
re := require.New(t)

numKeys := 10
actual := GenerateKeys(numKeys)
re.Len(actual, numKeys)

// make sure every key:
// i. has length `keyLen`
// ii. has only characters from `keyChars`
for _, key := range actual {
re.Len(key, keyLen)
for _, char := range key {
re.True(strings.ContainsRune(keyChars, char))
}
}
}

func TestGenerateSplitKey(t *testing.T) {
re := require.New(t)

// empty key
s := []byte("")
e := []byte{116, 128, 0, 0, 0, 0, 0, 0, 255, 1, 0, 0, 0, 0, 0, 0, 0, 248}
splitKey := GenerateSplitKey(s, e)
re.Less(string(s), string(splitKey))
re.Less(string(splitKey), string(e))

// empty end key
s = []byte{116, 128, 0, 0, 0, 0, 0, 0, 255, 1, 0, 0, 0, 0, 0, 0, 0, 248}
e = []byte("")
splitKey = GenerateSplitKey(s, e)
re.Less(string(s), string(splitKey))
re.Less(string(e), string(splitKey))

// empty start and end keys
s = []byte{}
e = []byte{}
splitKey = GenerateSplitKey(s, e)
re.Less(string(s), string(splitKey))
re.Less(string(e), string(splitKey))
var s []byte
var e []byte
splitKey, err := GenerateTiDBEncodedSplitKey(s, e)
re.NoError(err)

// same start and end keys
s = codec.EncodeBytes([]byte{116, 128, 0, 0, 0, 0, 0, 0, 255, 1, 0, 0, 0, 0, 0, 0, 0, 248})
e = codec.EncodeBytes([]byte{116, 128, 0, 0, 0, 0, 0, 0, 255, 1, 0, 0, 0, 0, 0, 0, 0, 248})
splitKey = GenerateSplitKey(s, e)
re.Greater(string(s), string(splitKey))
re.Greater(string(e), string(splitKey))
// left
leftSplit, err := GenerateTiDBEncodedSplitKey(s, splitKey)
re.NoError(err)
re.Less(string(leftSplit), string(splitKey))
rightSplit, err := GenerateTiDBEncodedSplitKey(splitKey, e)
re.NoError(err)
re.Less(string(splitKey), string(rightSplit))

s = codec.EncodeBytes([]byte{116, 128, 0, 0, 0, 0, 0, 0, 1})
e = codec.EncodeBytes([]byte{116, 128, 0, 0, 0, 0, 0, 0, 1, 1})
splitKey = GenerateSplitKey(s, e)
re.Greater(string(s), string(splitKey))
re.Less(string(splitKey), string(e))
meta := &metapb.Region{
Id: 0,
Peers: nil,
RegionEpoch: &metapb.RegionEpoch{ConfVer: 1, Version: 1},
}
meta.StartKey = s
meta.EndKey = leftSplit
region := core.NewRegionInfo(
meta,
nil,
core.SetApproximateSize(int64(1)),
core.SetApproximateKeys(int64(1)),
)

regionsInfo := core.NewRegionsInfo()
origin, overlaps, rangeChanged := regionsInfo.SetRegion(region)
regionsInfo.UpdateSubTree(region, origin, overlaps, rangeChanged)

getRegion := regionsInfo.GetRegionByKey([]byte("a"))
re.NotNil(getRegion)
}

0 comments on commit ab7f903

Please sign in to comment.