-
Notifications
You must be signed in to change notification settings - Fork 728
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
mcs: fix rule sync when meet "no rule left" and concurrency #7481
Closed
Closed
Changes from 6 commits
Commits
Show all changes
18 commits
Select commit
Hold shift + click to select a range
7866780
mcs: fix rule sync when meet "no rule left"
lhy1024 10af033
add some test
lhy1024 9e006d3
add TestConcurrency
lhy1024 d785df3
merge two watcher
lhy1024 23d6032
Merge branch 'master' into fix-rule-sync
lhy1024 bd5acaa
fix init
lhy1024 48afd51
address comments
lhy1024 e2a79c6
Merge branch 'master' into fix-rule-sync
lhy1024 ac72124
fix test
lhy1024 9f72fe8
Merge branch 'master' of github.com:tikv/pd into fix-rule-sync
lhy1024 8b020e9
Merge branch 'fix-rule-sync' of github.com:lhy1024/pd into fix-rule-sync
lhy1024 0c70123
make test stable
lhy1024 89ff1fe
Merge branch 'master' into fix-rule-sync
lhy1024 ff23490
fix data race in test
lhy1024 4702d2a
Merge branch 'fix-rule-sync' of github.com:lhy1024/pd into fix-rule-sync
lhy1024 13e49f8
Merge branch 'master' into fix-rule-sync
lhy1024 520fbcb
Merge branch 'master' into fix-rule-sync
lhy1024 4087916
Merge branch 'master' into fix-rule-sync
rleungx File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,6 +25,7 @@ | |
"github.com/tikv/pd/pkg/schedule/placement" | ||
"github.com/tikv/pd/pkg/storage/endpoint" | ||
"github.com/tikv/pd/pkg/utils/etcdutil" | ||
"github.com/tikv/pd/pkg/utils/syncutil" | ||
"go.etcd.io/etcd/clientv3" | ||
"go.etcd.io/etcd/mvcc/mvccpb" | ||
"go.uber.org/zap" | ||
|
@@ -36,6 +37,10 @@ | |
cancel context.CancelFunc | ||
wg sync.WaitGroup | ||
|
||
// ruleCommonPathPrefix: | ||
// - Key: /pd/{cluster_id}/rule | ||
// - Value: placement.Rule or placement.RuleGroup | ||
ruleCommonPathPrefix string | ||
// rulesPathPrefix: | ||
// - Key: /pd/{cluster_id}/rules/{group_id}-{rule_id} | ||
// - Value: placement.Rule | ||
|
@@ -60,8 +65,18 @@ | |
regionLabeler *labeler.RegionLabeler | ||
|
||
ruleWatcher *etcdutil.LoopWatcher | ||
rleungx marked this conversation as resolved.
Show resolved
Hide resolved
|
||
groupWatcher *etcdutil.LoopWatcher | ||
labelWatcher *etcdutil.LoopWatcher | ||
|
||
// pendingDeletion is a structure used to track the rules or rule groups that are marked for deletion. | ||
// If a rule or rule group cannot be deleted immediately due to the absence of rules, | ||
// it will be held here and removed later when a new rule or rule group put event allows for its deletion. | ||
pendingDeletion struct { | ||
syncutil.RWMutex | ||
// key: path, value: [groupID, ruleID] | ||
// The map 'kvs' holds the rules or rule groups that are pending deletion. | ||
// If a rule group needs to be deleted, the ruleID will be an empty string. | ||
kvs map[string][2]string | ||
} | ||
} | ||
|
||
// NewWatcher creates a new watcher to watch the Placement Rule change from PD API server. | ||
|
@@ -79,22 +94,25 @@ | |
ctx: ctx, | ||
cancel: cancel, | ||
rulesPathPrefix: endpoint.RulesPathPrefix(clusterID), | ||
ruleCommonPathPrefix: endpoint.RuleCommonPathPrefix(clusterID), | ||
ruleGroupPathPrefix: endpoint.RuleGroupPathPrefix(clusterID), | ||
regionLabelPathPrefix: endpoint.RegionLabelPathPrefix(clusterID), | ||
etcdClient: etcdClient, | ||
ruleStorage: ruleStorage, | ||
checkerController: checkerController, | ||
ruleManager: ruleManager, | ||
regionLabeler: regionLabeler, | ||
pendingDeletion: struct { | ||
syncutil.RWMutex | ||
kvs map[string][2]string | ||
}{ | ||
kvs: make(map[string][2]string), | ||
}, | ||
} | ||
err := rw.initializeRuleWatcher() | ||
if err != nil { | ||
return nil, err | ||
} | ||
err = rw.initializeGroupWatcher() | ||
if err != nil { | ||
return nil, err | ||
} | ||
err = rw.initializeRegionLabelWatcher() | ||
if err != nil { | ||
return nil, err | ||
|
@@ -103,85 +121,87 @@ | |
} | ||
|
||
func (rw *Watcher) initializeRuleWatcher() error { | ||
prefixToTrim := rw.rulesPathPrefix + "/" | ||
putFn := func(kv *mvccpb.KeyValue) error { | ||
log.Info("update placement rule", zap.String("key", string(kv.Key)), zap.String("value", string(kv.Value))) | ||
rule, err := placement.NewRuleFromJSON(kv.Value) | ||
if err != nil { | ||
return err | ||
} | ||
// Update the suspect key ranges in the checker. | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
if oldRule := rw.ruleManager.GetRule(rule.GroupID, rule.ID); oldRule != nil { | ||
rw.checkerController.AddSuspectKeyRange(oldRule.StartKey, oldRule.EndKey) | ||
err := func() error { | ||
if strings.HasPrefix(string(kv.Key), rw.rulesPathPrefix) { | ||
log.Info("update placement rule", zap.String("key", string(kv.Key)), zap.String("value", string(kv.Value))) | ||
rule, err := placement.NewRuleFromJSON(kv.Value) | ||
if err != nil { | ||
return err | ||
} | ||
// Update the suspect key ranges in the checker. | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
if oldRule := rw.ruleManager.GetRule(rule.GroupID, rule.ID); oldRule != nil { | ||
rw.checkerController.AddSuspectKeyRange(oldRule.StartKey, oldRule.EndKey) | ||
} | ||
return rw.ruleManager.SetRule(rule) | ||
} else if strings.HasPrefix(string(kv.Key), rw.ruleGroupPathPrefix) { | ||
log.Info("update placement rule group", zap.String("key", string(kv.Key)), zap.String("value", string(kv.Value))) | ||
ruleGroup, err := placement.NewRuleGroupFromJSON(kv.Value) | ||
if err != nil { | ||
return err | ||
} | ||
// Add all rule key ranges within the group to the suspect key ranges. | ||
for _, rule := range rw.ruleManager.GetRulesByGroup(ruleGroup.ID) { | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
} | ||
return rw.ruleManager.SetRuleGroup(ruleGroup) | ||
} else { | ||
log.Warn("unknown key when update placement rule", zap.String("key", string(kv.Key))) | ||
return nil | ||
} | ||
}() | ||
if err == nil && rw.hasPendingDeletion() { | ||
rw.tryFinishPendingDeletion() | ||
} | ||
return rw.ruleManager.SetRule(rule) | ||
return err | ||
} | ||
deleteFn := func(kv *mvccpb.KeyValue) error { | ||
key := string(kv.Key) | ||
log.Info("delete placement rule", zap.String("key", key)) | ||
ruleJSON, err := rw.ruleStorage.LoadRule(strings.TrimPrefix(key, prefixToTrim)) | ||
if err != nil { | ||
return err | ||
} | ||
rule, err := placement.NewRuleFromJSON([]byte(ruleJSON)) | ||
if err != nil { | ||
return err | ||
groupID, ruleID, err := func() (string, string, error) { | ||
if strings.HasPrefix(string(kv.Key), rw.rulesPathPrefix) { | ||
lhy1024 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
log.Info("delete placement rule", zap.String("key", key)) | ||
ruleJSON, err := rw.ruleStorage.LoadRule(strings.TrimPrefix(key, rw.rulesPathPrefix+"/")) | ||
if err != nil { | ||
return "", "", err | ||
} | ||
rule, err := placement.NewRuleFromJSON([]byte(ruleJSON)) | ||
if err != nil { | ||
return "", "", err | ||
} | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
return rule.GroupID, rule.ID, rw.ruleManager.DeleteRule(rule.GroupID, rule.ID) | ||
} else if strings.HasPrefix(string(kv.Key), rw.ruleGroupPathPrefix) { | ||
lhy1024 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
log.Info("delete placement rule group", zap.String("key", key)) | ||
trimmedKey := strings.TrimPrefix(key, rw.ruleGroupPathPrefix+"/") | ||
for _, rule := range rw.ruleManager.GetRulesByGroup(trimmedKey) { | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
} | ||
return trimmedKey, "", rw.ruleManager.DeleteRuleGroup(trimmedKey) | ||
} else { | ||
log.Warn("unknown key when delete placement rule", zap.String("key", string(kv.Key))) | ||
lhy1024 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
return "", "", nil | ||
} | ||
}() | ||
if err != nil && strings.Contains(err.Error(), "no rule left") && groupID != "" { | ||
rw.addPendingDeletion(key, groupID, ruleID) | ||
} | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
return rw.ruleManager.DeleteRule(rule.GroupID, rule.ID) | ||
return nil | ||
} | ||
postEventFn := func() error { | ||
return nil | ||
} | ||
rw.ruleWatcher = etcdutil.NewLoopWatcher( | ||
rw.ctx, &rw.wg, | ||
rw.etcdClient, | ||
"scheduling-rule-watcher", rw.rulesPathPrefix, | ||
"scheduling-rule-watcher", rw.ruleCommonPathPrefix, | ||
putFn, deleteFn, postEventFn, | ||
clientv3.WithPrefix(), | ||
) | ||
rw.ruleWatcher.StartWatchLoop() | ||
return rw.ruleWatcher.WaitLoad() | ||
} | ||
|
||
func (rw *Watcher) initializeGroupWatcher() error { | ||
prefixToTrim := rw.ruleGroupPathPrefix + "/" | ||
putFn := func(kv *mvccpb.KeyValue) error { | ||
log.Info("update placement rule group", zap.String("key", string(kv.Key)), zap.String("value", string(kv.Value))) | ||
ruleGroup, err := placement.NewRuleGroupFromJSON(kv.Value) | ||
if err != nil { | ||
return err | ||
} | ||
// Add all rule key ranges within the group to the suspect key ranges. | ||
for _, rule := range rw.ruleManager.GetRulesByGroup(ruleGroup.ID) { | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
} | ||
return rw.ruleManager.SetRuleGroup(ruleGroup) | ||
} | ||
deleteFn := func(kv *mvccpb.KeyValue) error { | ||
key := string(kv.Key) | ||
log.Info("delete placement rule group", zap.String("key", key)) | ||
trimmedKey := strings.TrimPrefix(key, prefixToTrim) | ||
for _, rule := range rw.ruleManager.GetRulesByGroup(trimmedKey) { | ||
rw.checkerController.AddSuspectKeyRange(rule.StartKey, rule.EndKey) | ||
} | ||
return rw.ruleManager.DeleteRuleGroup(trimmedKey) | ||
} | ||
postEventFn := func() error { | ||
return nil | ||
} | ||
rw.groupWatcher = etcdutil.NewLoopWatcher( | ||
rw.ctx, &rw.wg, | ||
rw.etcdClient, | ||
"scheduling-rule-group-watcher", rw.ruleGroupPathPrefix, | ||
putFn, deleteFn, postEventFn, | ||
clientv3.WithPrefix(), | ||
) | ||
rw.groupWatcher.StartWatchLoop() | ||
return rw.groupWatcher.WaitLoad() | ||
} | ||
|
||
func (rw *Watcher) initializeRegionLabelWatcher() error { | ||
prefixToTrim := rw.regionLabelPathPrefix + "/" | ||
putFn := func(kv *mvccpb.KeyValue) error { | ||
|
@@ -216,3 +236,39 @@ | |
rw.cancel() | ||
rw.wg.Wait() | ||
} | ||
|
||
func (rw *Watcher) hasPendingDeletion() bool { | ||
rw.pendingDeletion.RLock() | ||
defer rw.pendingDeletion.RUnlock() | ||
return len(rw.pendingDeletion.kvs) > 0 | ||
} | ||
|
||
func (rw *Watcher) addPendingDeletion(path, groupID, ruleID string) { | ||
rw.pendingDeletion.Lock() | ||
defer rw.pendingDeletion.Unlock() | ||
rw.pendingDeletion.kvs[path] = [2]string{groupID, ruleID} | ||
} | ||
|
||
func (rw *Watcher) tryFinishPendingDeletion() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am worried about if the put and delete can be disordered. If so, the newly added rule might be deleted unexpectedly. |
||
rw.pendingDeletion.Lock() | ||
defer rw.pendingDeletion.Unlock() | ||
originLen := len(rw.pendingDeletion.kvs) | ||
for k, v := range rw.pendingDeletion.kvs { | ||
groupID, ruleID := v[0], v[1] | ||
var err error | ||
if ruleID == "" { | ||
err = rw.ruleManager.DeleteRuleGroup(groupID) | ||
} else { | ||
err = rw.ruleManager.DeleteRule(groupID, ruleID) | ||
} | ||
if err == nil { | ||
delete(rw.pendingDeletion.kvs, k) | ||
} | ||
} | ||
// If the length of the map is changed, it means that some rules or rule groups have been deleted. | ||
// We need to force load the rules and rule groups to make sure sync with etcd. | ||
if len(rw.pendingDeletion.kvs) != originLen { | ||
rw.ruleWatcher.ForceLoad() | ||
log.Info("force load rules", zap.Int("pending deletion", len(rw.pendingDeletion.kvs)), zap.Int("origin", originLen)) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the reason for this modification? Do we need to modify the corresponding API interface for the non-API mode?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Avoid failure in JSON unmarshalling and maintain consistency with PD mode.