Skip to content

Commit

Permalink
Merge branch 'release-7.1' into cherry-pick-7087-to-release-7.1
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot[bot] authored Sep 28, 2023
2 parents c01e797 + e063180 commit 3864db7
Show file tree
Hide file tree
Showing 17 changed files with 293 additions and 32 deletions.
2 changes: 1 addition & 1 deletion pkg/mock/mockcluster/mockcluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,7 @@ func (mc *Cluster) AllocPeer(storeID uint64) (*metapb.Peer, error) {
func (mc *Cluster) initRuleManager() {
if mc.RuleManager == nil {
mc.RuleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), mc, mc.GetOpts())
mc.RuleManager.Initialize(int(mc.GetReplicationConfig().MaxReplicas), mc.GetReplicationConfig().LocationLabels)
mc.RuleManager.Initialize(int(mc.GetReplicationConfig().MaxReplicas), mc.GetReplicationConfig().LocationLabels, mc.GetReplicationConfig().IsolationLevel)
}
}

Expand Down
35 changes: 35 additions & 0 deletions pkg/schedule/checker/rule_checker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,41 @@ func (suite *ruleCheckerTestSuite) TestAddRulePeerWithIsolationLevel() {
suite.Equal(uint64(4), op.Step(0).(operator.AddLearner).ToStore)
}

func (suite *ruleCheckerTestSuite) TestReplaceDownPeerWithIsolationLevel() {
suite.cluster.SetMaxStoreDownTime(100 * time.Millisecond)
suite.cluster.AddLabelsStore(1, 1, map[string]string{"zone": "z1", "host": "h1"})
suite.cluster.AddLabelsStore(2, 1, map[string]string{"zone": "z1", "host": "h2"})
suite.cluster.AddLabelsStore(3, 1, map[string]string{"zone": "z2", "host": "h3"})
suite.cluster.AddLabelsStore(4, 1, map[string]string{"zone": "z2", "host": "h4"})
suite.cluster.AddLabelsStore(5, 1, map[string]string{"zone": "z3", "host": "h5"})
suite.cluster.AddLabelsStore(6, 1, map[string]string{"zone": "z3", "host": "h6"})
suite.cluster.AddLeaderRegionWithRange(1, "", "", 1, 3, 5)
suite.ruleManager.DeleteRule("pd", "default")
suite.ruleManager.SetRule(&placement.Rule{
GroupID: "pd",
ID: "test",
Index: 100,
Override: true,
Role: placement.Voter,
Count: 3,
LocationLabels: []string{"zone", "host"},
IsolationLevel: "zone",
})
op := suite.rc.Check(suite.cluster.GetRegion(1))
suite.Nil(op)
region := suite.cluster.GetRegion(1)
downPeer := []*pdpb.PeerStats{
{Peer: region.GetStorePeer(5), DownSeconds: 6000},
}
region = region.Clone(core.WithDownPeers(downPeer))
suite.cluster.PutRegion(region)
suite.cluster.SetStoreDown(5)
suite.cluster.SetStoreDown(6)
time.Sleep(200 * time.Millisecond)
op = suite.rc.Check(suite.cluster.GetRegion(1))
suite.Nil(op)
}

func (suite *ruleCheckerTestSuite) TestFixPeer() {
suite.cluster.AddLeaderStore(1, 1)
suite.cluster.AddLeaderStore(2, 1)
Expand Down
5 changes: 4 additions & 1 deletion pkg/schedule/placement/rule_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func NewRuleManager(storage endpoint.RuleStorage, storeSetInformer core.StoreSet

// Initialize loads rules from storage. If Placement Rules feature is never enabled, it creates default rule that is
// compatible with previous configuration.
func (m *RuleManager) Initialize(maxReplica int, locationLabels []string) error {
func (m *RuleManager) Initialize(maxReplica int, locationLabels []string, isolationLevel string) error {
m.Lock()
defer m.Unlock()
if m.initialized {
Expand All @@ -93,6 +93,7 @@ func (m *RuleManager) Initialize(maxReplica int, locationLabels []string) error
Role: Voter,
Count: maxReplica - witnessCount,
LocationLabels: locationLabels,
IsolationLevel: isolationLevel,
},
{
GroupID: "pd",
Expand All @@ -101,6 +102,7 @@ func (m *RuleManager) Initialize(maxReplica int, locationLabels []string) error
Count: witnessCount,
IsWitness: true,
LocationLabels: locationLabels,
IsolationLevel: isolationLevel,
},
}...,
)
Expand All @@ -111,6 +113,7 @@ func (m *RuleManager) Initialize(maxReplica int, locationLabels []string) error
Role: Voter,
Count: maxReplica,
LocationLabels: locationLabels,
IsolationLevel: isolationLevel,
})
}
for _, defaultRule := range defaultRules {
Expand Down
6 changes: 3 additions & 3 deletions pkg/schedule/placement/rule_manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func newTestManager(t *testing.T, enableWitness bool) (endpoint.RuleStorage, *Ru
var err error
manager := NewRuleManager(store, nil, mockconfig.NewTestOptions())
manager.conf.SetWitnessEnabled(enableWitness)
err = manager.Initialize(3, []string{"zone", "rack", "host"})
err = manager.Initialize(3, []string{"zone", "rack", "host"}, "")
re.NoError(err)
return store, manager
}
Expand Down Expand Up @@ -157,7 +157,7 @@ func TestSaveLoad(t *testing.T) {
}

m2 := NewRuleManager(store, nil, nil)
err := m2.Initialize(3, []string{"no", "labels"})
err := m2.Initialize(3, []string{"no", "labels"}, "")
re.NoError(err)
re.Len(m2.GetAllRules(), 3)
re.Equal(rules[0].String(), m2.GetRule("pd", "default").String())
Expand All @@ -173,7 +173,7 @@ func TestSetAfterGet(t *testing.T) {
manager.SetRule(rule)

m2 := NewRuleManager(store, nil, nil)
err := m2.Initialize(100, []string{})
err := m2.Initialize(100, []string{}, "")
re.NoError(err)
rule = m2.GetRule("pd", "default")
re.Equal(1, rule.Count)
Expand Down
13 changes: 6 additions & 7 deletions pkg/schedule/scheduler.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,17 +118,16 @@ func CreateScheduler(typ string, opController *OperatorController, storage endpo
if !ok {
return nil, errs.ErrSchedulerCreateFuncNotRegistered.FastGenByArgs(typ)
}
return fn(opController, storage, dec)
}

s, err := fn(opController, storage, dec)
if err != nil {
return nil, err
}
// SaveSchedulerConfig saves the config of the specified scheduler.
func SaveSchedulerConfig(storage endpoint.ConfigStorage, s Scheduler) error {
data, err := s.EncodeConfig()
if err != nil {
return nil, err
return err
}
err = storage.SaveScheduleConfig(s.GetName(), data)
return s, err
return storage.SaveScheduleConfig(s.GetName(), data)
}

// FindSchedulerTypeByName finds the type of the specified name.
Expand Down
2 changes: 1 addition & 1 deletion pkg/schedule/schedulers/evict_leader.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func newEvictLeaderScheduler(opController *schedule.OperatorController, conf *ev
}
}

// EvictStores returns the IDs of the evict-stores.
// EvictStoreIDs returns the IDs of the evict-stores.
func (s *evictLeaderScheduler) EvictStoreIDs() []uint64 {
return s.conf.getStores()
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/statistics/region_collection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ func TestRegionStatistics(t *testing.T) {
re := require.New(t)
store := storage.NewStorageWithMemoryBackend()
manager := placement.NewRuleManager(store, nil, nil)
err := manager.Initialize(3, []string{"zone", "rack", "host"})
err := manager.Initialize(3, []string{"zone", "rack", "host"}, "")
re.NoError(err)
opt := mockconfig.NewTestOptions()
opt.SetPlacementRuleEnabled(false)
Expand Down Expand Up @@ -135,7 +135,7 @@ func TestRegionStatisticsWithPlacementRule(t *testing.T) {
re := require.New(t)
store := storage.NewStorageWithMemoryBackend()
manager := placement.NewRuleManager(store, nil, nil)
err := manager.Initialize(3, []string{"zone", "rack", "host"})
err := manager.Initialize(3, []string{"zone", "rack", "host"}, "")
re.NoError(err)
opt := mockconfig.NewTestOptions()
opt.SetPlacementRuleEnabled(true)
Expand Down
4 changes: 3 additions & 1 deletion server/api/operator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -383,7 +383,9 @@ func (suite *transferRegionOperatorTestSuite) TestTransferRegionWithPlacementRul
if testCase.placementRuleEnable {
err := suite.svr.GetRaftCluster().GetRuleManager().Initialize(
suite.svr.GetRaftCluster().GetOpts().GetMaxReplicas(),
suite.svr.GetRaftCluster().GetOpts().GetLocationLabels())
suite.svr.GetRaftCluster().GetOpts().GetLocationLabels(),
suite.svr.GetRaftCluster().GetOpts().GetIsolationLevel(),
)
suite.NoError(err)
}
if len(testCase.rules) > 0 {
Expand Down
2 changes: 1 addition & 1 deletion server/cluster/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ func (c *RaftCluster) Start(s Server) error {
}
c.ruleManager = placement.NewRuleManager(c.storage, c, c.GetOpts())
if c.opt.IsPlacementRulesEnabled() {
err = c.ruleManager.Initialize(c.opt.GetMaxReplicas(), c.opt.GetLocationLabels())
err = c.ruleManager.Initialize(c.opt.GetMaxReplicas(), c.opt.GetLocationLabels(), c.opt.GetIsolationLevel())
if err != nil {
return err
}
Expand Down
10 changes: 5 additions & 5 deletions server/cluster/cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ func TestSetOfflineStore(t *testing.T) {
cluster.coordinator = newCoordinator(ctx, cluster, nil)
cluster.ruleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts())
if opt.IsPlacementRulesEnabled() {
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels())
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel())
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -428,7 +428,7 @@ func TestUpStore(t *testing.T) {
cluster.coordinator = newCoordinator(ctx, cluster, nil)
cluster.ruleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts())
if opt.IsPlacementRulesEnabled() {
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels())
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel())
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -531,7 +531,7 @@ func TestDeleteStoreUpdatesClusterVersion(t *testing.T) {
cluster.coordinator = newCoordinator(ctx, cluster, nil)
cluster.ruleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts())
if opt.IsPlacementRulesEnabled() {
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels())
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel())
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -1257,7 +1257,7 @@ func TestOfflineAndMerge(t *testing.T) {
cluster.coordinator = newCoordinator(ctx, cluster, nil)
cluster.ruleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts())
if opt.IsPlacementRulesEnabled() {
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels())
err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel())
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -2044,7 +2044,7 @@ func newTestRaftCluster(
rc.InitCluster(id, opt, s, basicCluster, nil)
rc.ruleManager = placement.NewRuleManager(storage.NewStorageWithMemoryBackend(), rc, opt)
if opt.IsPlacementRulesEnabled() {
err := rc.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels())
err := rc.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel())
if err != nil {
panic(err)
}
Expand Down
4 changes: 4 additions & 0 deletions server/cluster/coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -656,6 +656,10 @@ func (c *coordinator) addScheduler(scheduler schedule.Scheduler, args ...string)
c.wg.Add(1)
go c.runScheduler(s)
c.schedulers[s.GetName()] = s
if err := schedule.SaveSchedulerConfig(c.cluster.storage, scheduler); err != nil {
log.Error("can not save scheduler config", zap.String("scheduler-name", scheduler.GetName()), errs.ZapError(err))
return err
}
c.cluster.opt.AddSchedulerCfg(s.GetType(), args)
return nil
}
Expand Down
3 changes: 2 additions & 1 deletion server/cluster/coordinator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -831,8 +831,9 @@ func TestPersistScheduler(t *testing.T) {
// whether the schedulers added or removed in dynamic way are recorded in opt
_, newOpt, err := newTestScheduleConfig()
re.NoError(err)
_, err = schedule.CreateScheduler(schedulers.ShuffleRegionType, oc, storage, schedule.ConfigJSONDecoder([]byte("null")))
shuffle, err := schedule.CreateScheduler(schedulers.ShuffleRegionType, oc, storage, schedule.ConfigJSONDecoder([]byte("null")))
re.NoError(err)
re.NoError(co.addScheduler(shuffle))
// suppose we add a new default enable scheduler
config.DefaultSchedulers = append(config.DefaultSchedulers, config.SchedulerConfig{Type: "shuffle-region"})
defer func() {
Expand Down
7 changes: 7 additions & 0 deletions server/config/persist_options.go
Original file line number Diff line number Diff line change
Expand Up @@ -319,6 +319,13 @@ func (o *PersistOptions) SetEnableWitness(enable bool) {
o.SetScheduleConfig(v)
}

// SetMaxStoreDownTime to set the max store down time. It's only used to test.
func (o *PersistOptions) SetMaxStoreDownTime(time time.Duration) {
v := o.GetScheduleConfig().Clone()
v.MaxStoreDownTime = typeutil.NewDuration(time)
o.SetScheduleConfig(v)
}

// SetMaxMergeRegionSize sets the max merge region size.
func (o *PersistOptions) SetMaxMergeRegionSize(maxMergeRegionSize uint64) {
v := o.GetScheduleConfig().Clone()
Expand Down
11 changes: 6 additions & 5 deletions server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -970,7 +970,7 @@ func (s *Server) SetReplicationConfig(cfg config.ReplicationConfig) error {
}
if cfg.EnablePlacementRules {
// initialize rule manager.
if err := rc.GetRuleManager().Initialize(int(cfg.MaxReplicas), cfg.LocationLabels); err != nil {
if err := rc.GetRuleManager().Initialize(int(cfg.MaxReplicas), cfg.LocationLabels, cfg.IsolationLevel); err != nil {
return err
}
} else {
Expand All @@ -993,19 +993,19 @@ func (s *Server) SetReplicationConfig(cfg config.ReplicationConfig) error {
defaultRule := rc.GetRuleManager().GetRule("pd", "default")

CheckInDefaultRule := func() error {
// replication config won't work when placement rule is enabled and exceeds one default rule
// replication config won't work when placement rule is enabled and exceeds one default rule
if !(defaultRule != nil &&
len(defaultRule.StartKey) == 0 && len(defaultRule.EndKey) == 0) {
return errors.New("cannot update MaxReplicas or LocationLabels when placement rules feature is enabled and not only default rule exists, please update rule instead")
return errors.New("cannot update MaxReplicas, LocationLabels or IsolationLevel when placement rules feature is enabled and not only default rule exists, please update rule instead")
}
if !(defaultRule.Count == int(old.MaxReplicas) && typeutil.StringsEqual(defaultRule.LocationLabels, []string(old.LocationLabels))) {
if !(defaultRule.Count == int(old.MaxReplicas) && typeutil.StringsEqual(defaultRule.LocationLabels, []string(old.LocationLabels)) && defaultRule.IsolationLevel == old.IsolationLevel) {
return errors.New("cannot to update replication config, the default rules do not consistent with replication config, please update rule instead")
}

return nil
}

if !(cfg.MaxReplicas == old.MaxReplicas && typeutil.StringsEqual(cfg.LocationLabels, old.LocationLabels)) {
if !(cfg.MaxReplicas == old.MaxReplicas && typeutil.StringsEqual(cfg.LocationLabels, old.LocationLabels) && cfg.IsolationLevel == old.IsolationLevel) {
if err := CheckInDefaultRule(); err != nil {
return err
}
Expand All @@ -1016,6 +1016,7 @@ func (s *Server) SetReplicationConfig(cfg config.ReplicationConfig) error {
if rule != nil {
rule.Count = int(cfg.MaxReplicas)
rule.LocationLabels = cfg.LocationLabels
rule.IsolationLevel = cfg.IsolationLevel
rc := s.GetRaftCluster()
if rc == nil {
return errs.ErrNotBootstrapped.GenWithStackByArgs()
Expand Down
40 changes: 36 additions & 4 deletions tests/pdctl/config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -684,7 +684,7 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
re.Equal(expect, replicationCfg.MaxReplicas)
}

checkLocaltionLabels := func(expect int) {
checkLocationLabels := func(expect int) {
args := []string{"-u", pdAddr, "config", "show", "replication"}
output, err := pdctl.ExecuteCommand(cmd, args...)
re.NoError(err)
Expand All @@ -693,6 +693,15 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
re.Len(replicationCfg.LocationLabels, expect)
}

checkIsolationLevel := func(expect string) {
args := []string{"-u", pdAddr, "config", "show", "replication"}
output, err := pdctl.ExecuteCommand(cmd, args...)
re.NoError(err)
replicationCfg := config.ReplicationConfig{}
re.NoError(json.Unmarshal(output, &replicationCfg))
re.Equal(replicationCfg.IsolationLevel, expect)
}

checkRuleCount := func(expect int) {
args := []string{"-u", pdAddr, "config", "placement-rules", "show", "--group", "pd", "--id", "default"}
output, err := pdctl.ExecuteCommand(cmd, args...)
Expand All @@ -711,6 +720,15 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
re.Len(rule.LocationLabels, expect)
}

checkRuleIsolationLevel := func(expect string) {
args := []string{"-u", pdAddr, "config", "placement-rules", "show", "--group", "pd", "--id", "default"}
output, err := pdctl.ExecuteCommand(cmd, args...)
re.NoError(err)
rule := placement.Rule{}
re.NoError(json.Unmarshal(output, &rule))
re.Equal(rule.IsolationLevel, expect)
}

// update successfully when placement rules is not enabled.
output, err := pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "set", "max-replicas", "2")
re.NoError(err)
Expand All @@ -719,8 +737,13 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
output, err = pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "set", "location-labels", "zone,host")
re.NoError(err)
re.Contains(string(output), "Success!")
checkLocaltionLabels(2)
output, err = pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "set", "isolation-level", "zone")
re.NoError(err)
re.Contains(string(output), "Success!")
checkLocationLabels(2)
checkRuleLocationLabels(2)
checkIsolationLevel("zone")
checkRuleIsolationLevel("zone")

// update successfully when only one default rule exists.
output, err = pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "placement-rules", "enable")
Expand All @@ -733,11 +756,18 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
checkMaxReplicas(3)
checkRuleCount(3)

// We need to change isolation first because we will validate
// if the location label contains the isolation level when setting location labels.
output, err = pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "set", "isolation-level", "host")
re.NoError(err)
re.Contains(string(output), "Success!")
output, err = pdctl.ExecuteCommand(cmd, "-u", pdAddr, "config", "set", "location-labels", "host")
re.NoError(err)
re.Contains(string(output), "Success!")
checkLocaltionLabels(1)
checkLocationLabels(1)
checkRuleLocationLabels(1)
checkIsolationLevel("host")
checkRuleIsolationLevel("host")

// update unsuccessfully when many rule exists.
fname := t.TempDir()
Expand All @@ -761,8 +791,10 @@ func TestUpdateDefaultReplicaConfig(t *testing.T) {
re.NoError(err)
checkMaxReplicas(4)
checkRuleCount(4)
checkLocaltionLabels(1)
checkLocationLabels(1)
checkRuleLocationLabels(1)
checkIsolationLevel("host")
checkRuleIsolationLevel("host")
}

func TestPDServerConfig(t *testing.T) {
Expand Down
Loading

0 comments on commit 3864db7

Please sign in to comment.