From cfa263a665f56e9faa2b1289601c1ba3f1978a5a Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 13 Sep 2023 14:09:08 +0800 Subject: [PATCH 01/10] mcs/scheduling: register the scheduler handler in API service mode (#7082) ref tikv/pd#5839 To ensure that the HTTP API handler could be initialized properly for each scheduler in the API service mode, this PR updates the scheduler controller and PD server handler to support initialize the HTTP handler only. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/mcs/scheduling/server/config/config.go | 21 ----- pkg/mcs/scheduling/server/config/watcher.go | 16 +++- pkg/mcs/scheduling/server/server.go | 4 +- pkg/schedule/coordinator.go | 5 + pkg/schedule/schedulers/scheduler.go | 6 +- .../schedulers/scheduler_controller.go | 80 +++++++++++++--- server/cluster/cluster.go | 10 ++ server/handler.go | 33 ++++--- .../mcs/scheduling/config_test.go | 92 ++++++++++++++++--- tests/server/api/api_test.go | 9 +- tests/server/api/testutil.go | 70 ++++++++++++++ 11 files changed, 263 insertions(+), 83 deletions(-) create mode 100644 tests/server/api/testutil.go diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index e0e5bb9b661..5b80612744b 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -19,7 +19,6 @@ import ( "os" "path/filepath" "strings" - "sync" "sync/atomic" "time" "unsafe" @@ -204,8 +203,6 @@ type PersistConfig struct { schedule atomic.Value replication atomic.Value storeConfig atomic.Value - // Store the respective configurations for different schedulers. - schedulerConfig sync.Map } // NewPersistConfig creates a new PersistConfig instance. @@ -275,24 +272,6 @@ func (o *PersistConfig) GetStoreConfig() *sc.StoreConfig { return o.storeConfig.Load().(*sc.StoreConfig) } -// SetSchedulerConfig sets the scheduler configuration with the given name. -func (o *PersistConfig) SetSchedulerConfig(name, data string) { - o.schedulerConfig.Store(name, data) -} - -// RemoveSchedulerConfig removes the scheduler configuration with the given name. -func (o *PersistConfig) RemoveSchedulerConfig(name string) { - o.schedulerConfig.Delete(name) -} - -// GetSchedulerConfig returns the scheduler configuration with the given name. -func (o *PersistConfig) GetSchedulerConfig(name string) string { - if v, ok := o.schedulerConfig.Load(name); ok { - return v.(string) - } - return "" -} - // GetMaxReplicas returns the max replicas. func (o *PersistConfig) GetMaxReplicas() int { return int(o.GetReplicationConfig().MaxReplicas) diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index e6e204b8631..d65f1a6b553 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -23,6 +23,7 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/log" sc "github.com/tikv/pd/pkg/schedule/config" + "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "go.etcd.io/etcd/clientv3" @@ -50,6 +51,9 @@ type Watcher struct { schedulerConfigWatcher *etcdutil.LoopWatcher *PersistConfig + // Some data, like the scheduler configs, should be loaded into the storage + // to make sure the coordinator could access them correctly. + storage storage.Storage } type persistedConfig struct { @@ -65,6 +69,7 @@ func NewWatcher( etcdClient *clientv3.Client, clusterID uint64, persistConfig *PersistConfig, + storage storage.Storage, ) (*Watcher, error) { ctx, cancel := context.WithCancel(ctx) cw := &Watcher{ @@ -74,6 +79,7 @@ func NewWatcher( schedulerConfigPathPrefix: endpoint.SchedulerConfigPathPrefix(clusterID), etcdClient: etcdClient, PersistConfig: persistConfig, + storage: storage, } err := cw.initializeConfigWatcher() if err != nil { @@ -120,15 +126,15 @@ func (cw *Watcher) initializeConfigWatcher() error { func (cw *Watcher) initializeSchedulerConfigWatcher() error { prefixToTrim := cw.schedulerConfigPathPrefix + "/" putFn := func(kv *mvccpb.KeyValue) error { - cw.SetSchedulerConfig( + return cw.storage.SaveScheduleConfig( strings.TrimPrefix(string(kv.Key), prefixToTrim), - string(kv.Value), + kv.Value, ) - return nil } deleteFn := func(kv *mvccpb.KeyValue) error { - cw.RemoveSchedulerConfig(strings.TrimPrefix(string(kv.Key), prefixToTrim)) - return nil + return cw.storage.RemoveScheduleConfig( + strings.TrimPrefix(string(kv.Key), prefixToTrim), + ) } postEventFn := func() error { return nil diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 1e3aea41aa5..8a74abf4b5e 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -432,11 +432,11 @@ func (s *Server) startServer() (err error) { func (s *Server) startCluster(context.Context) error { s.basicCluster = core.NewBasicCluster() + s.storage = endpoint.NewStorageEndpoint(kv.NewMemoryKV(), nil) err := s.startWatcher() if err != nil { return err } - s.storage = endpoint.NewStorageEndpoint(kv.NewMemoryKV(), nil) s.hbStreams = hbstream.NewHeartbeatStreams(s.Context(), s.clusterID, s.basicCluster) s.cluster, err = NewCluster(s.Context(), s.persistConfig, s.storage, s.basicCluster, s.hbStreams, s.clusterID, s.checkMembershipCh) if err != nil { @@ -458,7 +458,7 @@ func (s *Server) startWatcher() (err error) { if err != nil { return err } - s.configWatcher, err = config.NewWatcher(s.Context(), s.GetClient(), s.clusterID, s.persistConfig) + s.configWatcher, err = config.NewWatcher(s.Context(), s.GetClient(), s.clusterID, s.persistConfig, s.storage) if err != nil { return err } diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 7e21919b214..ab0500d0445 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -444,6 +444,8 @@ func (c *Coordinator) InitSchedulers(needRun bool) { if err = c.schedulers.AddScheduler(s); err != nil { log.Error("can not add scheduler with independent configuration", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", cfg.Args), errs.ZapError(err)) } + } else if err = c.schedulers.AddSchedulerHandler(s); err != nil { + log.Error("can not add scheduler handler with independent configuration", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", cfg.Args), errs.ZapError(err)) } } @@ -472,6 +474,8 @@ func (c *Coordinator) InitSchedulers(needRun bool) { scheduleCfg.Schedulers[k] = schedulerCfg k++ } + } else if err = c.schedulers.AddSchedulerHandler(s, schedulerCfg.Args...); err != nil && !errors.ErrorEqual(err, errs.ErrSchedulerExisted.FastGenByArgs()) { + log.Error("can not add scheduler handler", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", schedulerCfg.Args), errs.ZapError(err)) } } @@ -507,6 +511,7 @@ func (c *Coordinator) LoadPlugin(pluginPath string, ch chan string) { return } log.Info("create scheduler", zap.String("scheduler-name", s.GetName())) + // TODO: handle the plugin in API service mode. if err = c.schedulers.AddScheduler(s); err != nil { log.Error("can't add scheduler", zap.String("scheduler-name", s.GetName()), errs.ZapError(err)) return diff --git a/pkg/schedule/schedulers/scheduler.go b/pkg/schedule/schedulers/scheduler.go index b4c425047cd..909acb3494c 100644 --- a/pkg/schedule/schedulers/scheduler.go +++ b/pkg/schedule/schedulers/scheduler.go @@ -91,8 +91,10 @@ func ConfigSliceDecoder(name string, args []string) ConfigDecoder { // CreateSchedulerFunc is for creating scheduler. type CreateSchedulerFunc func(opController *operator.Controller, storage endpoint.ConfigStorage, dec ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) -var schedulerMap = make(map[string]CreateSchedulerFunc) -var schedulerArgsToDecoder = make(map[string]ConfigSliceDecoderBuilder) +var ( + schedulerMap = make(map[string]CreateSchedulerFunc) + schedulerArgsToDecoder = make(map[string]ConfigSliceDecoderBuilder) +) // RegisterScheduler binds a scheduler creator. It should be called in init() // func of a package. diff --git a/pkg/schedule/schedulers/scheduler_controller.go b/pkg/schedule/schedulers/scheduler_controller.go index 1c6329fb0b1..c8f07f56678 100644 --- a/pkg/schedule/schedulers/scheduler_controller.go +++ b/pkg/schedule/schedulers/scheduler_controller.go @@ -40,22 +40,28 @@ var denySchedulersByLabelerCounter = labeler.LabelerEventCounter.WithLabelValues // Controller is used to manage all schedulers. type Controller struct { sync.RWMutex - wg sync.WaitGroup - ctx context.Context - cluster sche.SchedulerCluster - storage endpoint.ConfigStorage - schedulers map[string]*ScheduleController - opController *operator.Controller + wg sync.WaitGroup + ctx context.Context + cluster sche.SchedulerCluster + storage endpoint.ConfigStorage + // schedulers is used to manage all schedulers, which will only be initialized + // and used in the PD leader service mode now. + schedulers map[string]*ScheduleController + // schedulerHandlers is used to manage the HTTP handlers of schedulers, + // which will only be initialized and used in the API service mode now. + schedulerHandlers map[string]http.Handler + opController *operator.Controller } // NewController creates a scheduler controller. func NewController(ctx context.Context, cluster sche.SchedulerCluster, storage endpoint.ConfigStorage, opController *operator.Controller) *Controller { return &Controller{ - ctx: ctx, - cluster: cluster, - storage: storage, - schedulers: make(map[string]*ScheduleController), - opController: opController, + ctx: ctx, + cluster: cluster, + storage: storage, + schedulers: make(map[string]*ScheduleController), + schedulerHandlers: make(map[string]http.Handler), + opController: opController, } } @@ -86,6 +92,9 @@ func (c *Controller) GetSchedulerNames() []string { func (c *Controller) GetSchedulerHandlers() map[string]http.Handler { c.RLock() defer c.RUnlock() + if len(c.schedulerHandlers) > 0 { + return c.schedulerHandlers + } handlers := make(map[string]http.Handler, len(c.schedulers)) for name, scheduler := range c.schedulers { handlers[name] = scheduler.Scheduler @@ -117,6 +126,50 @@ func (c *Controller) ResetSchedulerMetrics() { schedulerStatusGauge.Reset() } +// AddSchedulerHandler adds the HTTP handler for a scheduler. +func (c *Controller) AddSchedulerHandler(scheduler Scheduler, args ...string) error { + c.Lock() + defer c.Unlock() + + name := scheduler.GetName() + if _, ok := c.schedulerHandlers[name]; ok { + return errs.ErrSchedulerExisted.FastGenByArgs() + } + + c.schedulerHandlers[name] = scheduler + c.cluster.GetSchedulerConfig().AddSchedulerCfg(scheduler.GetType(), args) + return nil +} + +// RemoveSchedulerHandler removes the HTTP handler for a scheduler. +func (c *Controller) RemoveSchedulerHandler(name string) error { + c.Lock() + defer c.Unlock() + if c.cluster == nil { + return errs.ErrNotBootstrapped.FastGenByArgs() + } + s, ok := c.schedulerHandlers[name] + if !ok { + return errs.ErrSchedulerNotFound.FastGenByArgs() + } + + conf := c.cluster.GetSchedulerConfig() + conf.RemoveSchedulerCfg(s.(Scheduler).GetType()) + if err := conf.Persist(c.storage); err != nil { + log.Error("the option can not persist scheduler config", errs.ZapError(err)) + return err + } + + if err := c.storage.RemoveScheduleConfig(name); err != nil { + log.Error("can not remove the scheduler config", errs.ZapError(err)) + return err + } + + delete(c.schedulerHandlers, name) + + return nil +} + // AddScheduler adds a scheduler. func (c *Controller) AddScheduler(scheduler Scheduler, args ...string) error { c.Lock() @@ -249,8 +302,9 @@ func (c *Controller) IsSchedulerExisted(name string) (bool, error) { if c.cluster == nil { return false, errs.ErrNotBootstrapped.FastGenByArgs() } - _, ok := c.schedulers[name] - if !ok { + _, existScheduler := c.schedulers[name] + _, existHandler := c.schedulerHandlers[name] + if !existScheduler && !existHandler { return false, errs.ErrSchedulerNotFound.FastGenByArgs() } return true, nil diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index b16de73f84e..18bf66b8188 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -795,6 +795,16 @@ func (c *RaftCluster) GetSchedulerHandlers() map[string]http.Handler { return c.coordinator.GetSchedulersController().GetSchedulerHandlers() } +// AddSchedulerHandler adds a scheduler handler. +func (c *RaftCluster) AddSchedulerHandler(scheduler schedulers.Scheduler, args ...string) error { + return c.coordinator.GetSchedulersController().AddSchedulerHandler(scheduler, args...) +} + +// RemoveSchedulerHandler removes a scheduler handler. +func (c *RaftCluster) RemoveSchedulerHandler(name string) error { + return c.coordinator.GetSchedulersController().RemoveSchedulerHandler(name) +} + // AddScheduler adds a scheduler. func (c *RaftCluster) AddScheduler(scheduler schedulers.Scheduler, args ...string) error { return c.coordinator.GetSchedulersController().AddScheduler(scheduler, args...) diff --git a/server/handler.go b/server/handler.go index adc1e8ecd31..a90f8e3f04f 100644 --- a/server/handler.go +++ b/server/handler.go @@ -236,19 +236,24 @@ func (h *Handler) AddScheduler(name string, args ...string) error { return err } log.Info("create scheduler", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", args)) - if !h.s.IsAPIServiceMode() { + if h.s.IsAPIServiceMode() { + if err = c.AddSchedulerHandler(s, args...); err != nil { + log.Error("can not add scheduler handler", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", args), errs.ZapError(err)) + return err + } + log.Info("add scheduler handler successfully", zap.String("scheduler-name", name), zap.Strings("scheduler-args", args)) + } else { if err = c.AddScheduler(s, args...); err != nil { log.Error("can not add scheduler", zap.String("scheduler-name", s.GetName()), zap.Strings("scheduler-args", args), errs.ZapError(err)) return err } - } else { - c.GetSchedulerConfig().AddSchedulerCfg(s.GetType(), args) + log.Info("add scheduler successfully", zap.String("scheduler-name", name), zap.Strings("scheduler-args", args)) } if err = h.opt.Persist(c.GetStorage()); err != nil { log.Error("can not persist scheduler config", errs.ZapError(err)) return err } - log.Info("add scheduler successfully", zap.String("scheduler-name", name), zap.Strings("scheduler-args", args)) + log.Info("persist scheduler config successfully", zap.String("scheduler-name", name), zap.Strings("scheduler-args", args)) return nil } @@ -258,24 +263,18 @@ func (h *Handler) RemoveScheduler(name string) error { if err != nil { return err } - if !h.s.IsAPIServiceMode() { + if h.s.IsAPIServiceMode() { + if err = c.RemoveSchedulerHandler(name); err != nil { + log.Error("can not remove scheduler handler", zap.String("scheduler-name", name), errs.ZapError(err)) + } else { + log.Info("remove scheduler handler successfully", zap.String("scheduler-name", name)) + } + } else { if err = c.RemoveScheduler(name); err != nil { log.Error("can not remove scheduler", zap.String("scheduler-name", name), errs.ZapError(err)) } else { log.Info("remove scheduler successfully", zap.String("scheduler-name", name)) } - } else { - conf := c.GetSchedulerConfig() - c.GetSchedulerConfig().RemoveSchedulerCfg(schedulers.FindSchedulerTypeByName(name)) - if err := conf.Persist(c.GetStorage()); err != nil { - log.Error("the option can not persist scheduler config", errs.ZapError(err)) - return err - } - - if err := c.GetStorage().RemoveScheduleConfig(name); err != nil { - log.Error("can not remove the scheduler config", errs.ZapError(err)) - return err - } } return err } diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index 032cb4ad7ae..14c99baa6f5 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -16,16 +16,24 @@ package scheduling import ( "context" + "fmt" "testing" + "time" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/schedulers" + "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/storage/endpoint" + "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/tests" + "github.com/tikv/pd/tests/server/api" ) type configTestSuite struct { @@ -77,6 +85,7 @@ func (suite *configTestSuite) TestConfigWatch() { suite.pdLeaderServer.GetEtcdClient(), suite.cluster.GetCluster().GetId(), config.NewPersistConfig(config.NewConfig()), + endpoint.NewStorageEndpoint(kv.NewMemoryKV(), nil), ) re.NoError(err) // Check the initial config value. @@ -128,39 +137,92 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { // Make sure the config is persisted before the watcher is created. persistConfig(re, suite.pdLeaderServer) // Create a config watcher. + storage := endpoint.NewStorageEndpoint(kv.NewMemoryKV(), nil) watcher, err := config.NewWatcher( suite.ctx, suite.pdLeaderServer.GetEtcdClient(), suite.cluster.GetCluster().GetId(), config.NewPersistConfig(config.NewConfig()), + storage, ) re.NoError(err) // Get all default scheduler names. - var schedulerNames, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllScheduleConfig() - + var namesFromAPIServer, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllScheduleConfig() testutil.Eventually(re, func() bool { - targetCount := len(sc.DefaultSchedulers) - return len(schedulerNames) == targetCount + return len(namesFromAPIServer) == len(sc.DefaultSchedulers) }) // Check all default schedulers' configs. - for _, schedulerName := range schedulerNames { - testutil.Eventually(re, func() bool { - return len(watcher.GetSchedulerConfig(schedulerName)) > 0 - }) - } + var namesFromSchedulingServer []string + testutil.Eventually(re, func() bool { + namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + re.NoError(err) + return len(namesFromSchedulingServer) == len(namesFromAPIServer) + }) + re.Equal(namesFromAPIServer, namesFromSchedulingServer) // Add a new scheduler. - err = suite.pdLeaderServer.GetServer().GetHandler().AddEvictLeaderScheduler(1) - re.NoError(err) + api.MustAddScheduler(re, suite.pdLeaderServer.GetAddr(), schedulers.EvictLeaderName, map[string]interface{}{ + "store_id": 1, + }) // Check the new scheduler's config. testutil.Eventually(re, func() bool { - return len(watcher.GetSchedulerConfig(schedulers.EvictLeaderName)) > 0 + namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + re.NoError(err) + return slice.Contains(namesFromSchedulingServer, schedulers.EvictLeaderName) }) - // Remove the scheduler. - err = suite.pdLeaderServer.GetServer().GetHandler().RemoveScheduler(schedulers.EvictLeaderName) + // Update the scheduler by adding a store. + err = suite.pdLeaderServer.GetServer().GetRaftCluster().PutStore( + &metapb.Store{ + Id: 2, + Address: "mock://2", + State: metapb.StoreState_Up, + NodeState: metapb.NodeState_Serving, + LastHeartbeat: time.Now().UnixNano(), + Version: "4.0.0", + }, + ) re.NoError(err) + api.MustAddScheduler(re, suite.pdLeaderServer.GetAddr(), schedulers.EvictLeaderName, map[string]interface{}{ + "store_id": 2, + }) + assertEvictLeaderStoreIDs(re, storage, []uint64{1, 2}) + // Update the scheduler by removing a store. + api.MustDeleteScheduler(re, suite.pdLeaderServer.GetAddr(), fmt.Sprintf("%s-%d", schedulers.EvictLeaderName, 1)) + assertEvictLeaderStoreIDs(re, storage, []uint64{2}) + // Delete the scheduler. + api.MustDeleteScheduler(re, suite.pdLeaderServer.GetAddr(), schedulers.EvictLeaderName) // Check the removed scheduler's config. testutil.Eventually(re, func() bool { - return len(watcher.GetSchedulerConfig(schedulers.EvictLeaderName)) == 0 + namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + re.NoError(err) + return !slice.Contains(namesFromSchedulingServer, schedulers.EvictLeaderName) }) watcher.Close() } + +func assertEvictLeaderStoreIDs( + re *require.Assertions, storage *endpoint.StorageEndpoint, storeIDs []uint64, +) { + var ( + namesFromSchedulingServer, configs []string + err error + evictLeaderCfg struct { + StoreIDWithRanges map[uint64][]core.KeyRange `json:"store-id-ranges"` + } + ) + testutil.Eventually(re, func() bool { + namesFromSchedulingServer, configs, err = storage.LoadAllScheduleConfig() + re.NoError(err) + for idx, name := range namesFromSchedulingServer { + if name == schedulers.EvictLeaderName { + err = schedulers.DecodeConfig([]byte(configs[idx]), &evictLeaderCfg) + re.NoError(err) + return len(evictLeaderCfg.StoreIDWithRanges) == len(storeIDs) + } + } + return false + }) + // Validate the updated scheduler's config. + for _, storeID := range storeIDs { + re.Contains(evictLeaderCfg.StoreIDWithRanges, storeID) + } +} diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 61d47d7790c..da86a872045 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package api_test +package api import ( "bytes" @@ -44,13 +44,6 @@ import ( "go.uber.org/goleak" ) -// dialClient used to dial http request. -var dialClient = &http.Client{ - Transport: &http.Transport{ - DisableKeepAlives: true, - }, -} - func TestMain(m *testing.M) { goleak.VerifyTestMain(m, testutil.LeakOptions...) } diff --git a/tests/server/api/testutil.go b/tests/server/api/testutil.go new file mode 100644 index 00000000000..7a33bf39048 --- /dev/null +++ b/tests/server/api/testutil.go @@ -0,0 +1,70 @@ +// Copyright 2023 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package api + +import ( + "bytes" + "encoding/json" + "fmt" + "io" + "net/http" + + "github.com/stretchr/testify/require" +) + +const schedulersPrefix = "/pd/api/v1/schedulers" + +// dialClient used to dial http request. +var dialClient = &http.Client{ + Transport: &http.Transport{ + DisableKeepAlives: true, + }, +} + +// MustAddScheduler adds a scheduler with HTTP API. +func MustAddScheduler( + re *require.Assertions, serverAddr string, + schedulerName string, args map[string]interface{}, +) { + request := map[string]interface{}{ + "name": schedulerName, + } + for arg, val := range args { + request[arg] = val + } + data, err := json.Marshal(request) + re.NoError(err) + httpReq, err := http.NewRequest(http.MethodPost, fmt.Sprintf("%s%s", serverAddr, schedulersPrefix), bytes.NewBuffer(data)) + re.NoError(err) + // Send request. + resp, err := dialClient.Do(httpReq) + re.NoError(err) + defer resp.Body.Close() + data, err = io.ReadAll(resp.Body) + re.NoError(err) + re.Equal(http.StatusOK, resp.StatusCode, string(data)) +} + +// MustDeleteScheduler deletes a scheduler with HTTP API. +func MustDeleteScheduler(re *require.Assertions, serverAddr, schedulerName string) { + httpReq, err := http.NewRequest(http.MethodDelete, fmt.Sprintf("%s%s/%s", serverAddr, schedulersPrefix, schedulerName), nil) + re.NoError(err) + resp, err := dialClient.Do(httpReq) + re.NoError(err) + defer resp.Body.Close() + data, err := io.ReadAll(resp.Body) + re.NoError(err) + re.Equal(http.StatusOK, resp.StatusCode, string(data)) +} From d0cebac09f2967348578b5159d3436f74dd92266 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 13 Sep 2023 17:18:09 +0800 Subject: [PATCH 02/10] pd-api-bench: fix the gosec check (#7086) ref tikv/pd#4399 Fix the gosec check warning. Signed-off-by: JmPotato --- tools/pd-api-bench/main.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tools/pd-api-bench/main.go b/tools/pd-api-bench/main.go index 3b413d4dd76..722c7eb08fa 100644 --- a/tools/pd-api-bench/main.go +++ b/tools/pd-api-bench/main.go @@ -154,13 +154,13 @@ func main() { log.Println("concurrency == 0, exit") return } - pdClis := make([]pd.Client, 0) + pdClis := make([]pd.Client, *client) for i := 0; i < *client; i++ { - pdClis = append(pdClis, newPDClient(ctx)) + pdClis[i] = newPDClient(ctx) } - httpClis := make([]*http.Client, 0) + httpClis := make([]*http.Client, *client) for i := 0; i < *client; i++ { - httpClis = append(httpClis, newHTTPClient()) + httpClis[i] = newHTTPClient() } err = cases.InitCluster(ctx, pdClis[0], httpClis[0]) if err != nil { From 11b752e0bfdf0167029667a33bfc5397f311db38 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Thu, 14 Sep 2023 02:15:08 +0800 Subject: [PATCH 03/10] scatter: fix the unexpected error code (#7075) close tikv/pd#7073 scatter: fix the unexpected error code Signed-off-by: nolouch Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- errors.toml | 5 ++++ pkg/errs/errno.go | 1 + pkg/schedule/scatter/region_scatterer.go | 14 +++++----- pkg/schedule/scatter/region_scatterer_test.go | 26 ++++++++++++++----- server/grpc_service.go | 19 +++++--------- 5 files changed, 39 insertions(+), 26 deletions(-) diff --git a/errors.toml b/errors.toml index 95324df1888..a0040195e5d 100644 --- a/errors.toml +++ b/errors.toml @@ -111,6 +111,11 @@ error = ''' failed to get the source store ''' +["PD:common:ErrGetTargetStore"] +error = ''' +failed to get the target store +''' + ["PD:common:ErrIncorrectSystemTime"] error = ''' incorrect system time diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 91cd4a78c4f..a077751f561 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -34,6 +34,7 @@ const ( // common error in multiple packages var ( ErrGetSourceStore = errors.Normalize("failed to get the source store", errors.RFCCodeText("PD:common:ErrGetSourceStore")) + ErrGetTargetStore = errors.Normalize("failed to get the target store", errors.RFCCodeText("PD:common:ErrGetTargetStore")) ErrIncorrectSystemTime = errors.Normalize("incorrect system time", errors.RFCCodeText("PD:common:ErrIncorrectSystemTime")) ) diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index a676afca6cf..c47bcd27e91 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -284,10 +284,10 @@ func (r *RegionScatterer) Scatter(region *core.RegionInfo, group string, skipSto return nil, errors.Errorf("region %d is hot", region.GetID()) } - return r.scatterRegion(region, group, skipStoreLimit), nil + return r.scatterRegion(region, group, skipStoreLimit) } -func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, skipStoreLimit bool) *operator.Operator { +func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, skipStoreLimit bool) (*operator.Operator, error) { engineFilter := filter.NewEngineFilter(r.name, filter.NotSpecialEngines) ordinaryPeers := make(map[uint64]*metapb.Peer, len(region.GetPeers())) specialPeers := make(map[string]map[uint64]*metapb.Peer) @@ -296,7 +296,7 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, s for _, peer := range region.GetPeers() { store := r.cluster.GetStore(peer.GetStoreId()) if store == nil { - return nil + return nil, errs.ErrGetSourceStore.FastGenByArgs(fmt.Sprintf("store not found, peer: %v, region id: %d", peer, region.GetID())) } if engineFilter.Target(r.cluster.GetSharedConfig(), store).IsOK() { ordinaryPeers[peer.GetStoreId()] = peer @@ -358,7 +358,7 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, s targetLeader, leaderStorePickedCount := r.selectAvailableLeaderStore(group, region, leaderCandidateStores, r.ordinaryEngine) if targetLeader == 0 { scatterSkipNoLeaderCounter.Inc() - return nil + return nil, errs.ErrGetTargetStore.FastGenByArgs(fmt.Sprintf("no target leader store found, region: %v", region)) } for engine, peers := range specialPeers { @@ -375,7 +375,7 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, s if isSameDistribution(region, targetPeers, targetLeader) { scatterUnnecessaryCounter.Inc() r.Put(targetPeers, targetLeader, group) - return nil + return nil, nil } op, err := operator.CreateScatterRegionOperator("scatter-region", r.cluster, region, targetPeers, targetLeader, skipStoreLimit) if err != nil { @@ -385,7 +385,7 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, s } r.Put(targetPeers, region.GetLeader().GetStoreId(), group) log.Debug("fail to create scatter region operator", errs.ZapError(err)) - return nil + return nil, errs.ErrCreateOperator.FastGenByArgs(fmt.Sprintf("failed to create scatter region operator for region %v", region.GetID())) } if op != nil { scatterSuccessCounter.Inc() @@ -394,7 +394,7 @@ func (r *RegionScatterer) scatterRegion(region *core.RegionInfo, group string, s op.AdditionalInfos["leader-picked-count"] = strconv.FormatUint(leaderStorePickedCount, 10) op.SetPriorityLevel(constant.High) } - return op + return op, nil } func allowLeader(fit *placement.RegionFit, peer *metapb.Peer) bool { diff --git a/pkg/schedule/scatter/region_scatterer_test.go b/pkg/schedule/scatter/region_scatterer_test.go index 5bdbdcbf159..0fc7f0967d7 100644 --- a/pkg/schedule/scatter/region_scatterer_test.go +++ b/pkg/schedule/scatter/region_scatterer_test.go @@ -105,12 +105,18 @@ func scatter(re *require.Assertions, numStores, numRegions uint64, useRules bool tc.AddLeaderRegion(i, 1, 2, 3) } scatterer := NewRegionScatterer(ctx, tc, oc, tc.AddSuspectRegions) - + noNeedMoveNum := 0 for i := uint64(1); i <= numRegions; i++ { region := tc.GetRegion(i) - if op, _ := scatterer.Scatter(region, "", false); op != nil { + if op, err := scatterer.Scatter(region, "", false); err == nil { + if op == nil { + noNeedMoveNum++ + continue + } checkOperator(re, op) operator.ApplyOperator(tc, op) + } else { + re.Nil(op) } } @@ -140,6 +146,7 @@ func scatter(re *require.Assertions, numStores, numRegions uint64, useRules bool re.LessOrEqual(float64(count), 1.1*float64(numRegions)/float64(numStores)) re.GreaterOrEqual(float64(count), 0.9*float64(numRegions)/float64(numStores)) } + re.GreaterOrEqual(noNeedMoveNum, 0) } func scatterSpecial(re *require.Assertions, numOrdinaryStores, numSpecialStores, numRegions uint64) { @@ -651,7 +658,8 @@ func TestSelectedStoresTooFewPeers(t *testing.T) { // Try to scatter a region with peer store id 2/3/4 for i := uint64(1); i < 20; i++ { region := tc.AddLeaderRegion(i+200, i%3+2, (i+1)%3+2, (i+2)%3+2) - op := scatterer.scatterRegion(region, group, false) + op, err := scatterer.scatterRegion(region, group, false) + re.NoError(err) re.False(isPeerCountChanged(op)) if op != nil { re.Equal(group, op.AdditionalInfos["group"]) @@ -691,7 +699,8 @@ func TestSelectedStoresTooManyPeers(t *testing.T) { // test region with peer 1 2 3 for i := uint64(1); i < 20; i++ { region := tc.AddLeaderRegion(i+200, i%3+1, (i+1)%3+1, (i+2)%3+1) - op := scatterer.scatterRegion(region, group, false) + op, err := scatterer.scatterRegion(region, group, false) + re.NoError(err) re.False(isPeerCountChanged(op)) } } @@ -715,7 +724,8 @@ func TestBalanceLeader(t *testing.T) { scatterer := NewRegionScatterer(ctx, tc, oc, tc.AddSuspectRegions) for i := uint64(1001); i <= 1300; i++ { region := tc.AddLeaderRegion(i, 2, 3, 4) - op := scatterer.scatterRegion(region, group, false) + op, err := scatterer.scatterRegion(region, group, false) + re.NoError(err) re.False(isPeerCountChanged(op)) } // all leader will be balanced in three stores. @@ -745,7 +755,8 @@ func TestBalanceRegion(t *testing.T) { scatterer := NewRegionScatterer(ctx, tc, oc, tc.AddSuspectRegions) for i := uint64(1001); i <= 1300; i++ { region := tc.AddLeaderRegion(i, 2, 4, 6) - op := scatterer.scatterRegion(region, group, false) + op, err := scatterer.scatterRegion(region, group, false) + re.NoError(err) re.False(isPeerCountChanged(op)) } for i := uint64(2); i <= 7; i++ { @@ -754,7 +765,8 @@ func TestBalanceRegion(t *testing.T) { // Test for unhealthy region // ref https://github.com/tikv/pd/issues/6099 region := tc.AddLeaderRegion(1500, 2, 3, 4, 6) - op := scatterer.scatterRegion(region, group, false) + op, err := scatterer.scatterRegion(region, group, false) + re.NoError(err) re.False(isPeerCountChanged(op)) } diff --git a/server/grpc_service.go b/server/grpc_service.go index 973c45a622f..1d20bb22d4d 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -1709,18 +1709,13 @@ func (s *GrpcServer) ScatterRegion(ctx context.Context, request *pdpb.ScatterReg return nil, err } - if op == nil { - return &pdpb.ScatterRegionResponse{ - Header: s.wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, - "operator could not be allocated"), - }, nil - } - - if !rc.GetOperatorController().AddOperator(op) { - return &pdpb.ScatterRegionResponse{ - Header: s.wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, - "operator cancelled because store limit exceeded"), - }, nil + if op != nil { + if !rc.GetOperatorController().AddOperator(op) { + return &pdpb.ScatterRegionResponse{ + Header: s.wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, + "operator canceled because cannot add an operator to the execute queue"), + }, nil + } } return &pdpb.ScatterRegionResponse{ From 1a0c8fb726d5024f9a5be28ac196495315a18d40 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 14 Sep 2023 11:36:08 +0800 Subject: [PATCH 04/10] mcs: forward store requests to scheduling server (#6976) ref tikv/pd#5839 Signed-off-by: Ryan Leung --- go.mod | 12 +-- go.sum | 4 +- pkg/mcs/scheduling/server/cluster.go | 77 ++++++++++++++++++- pkg/mcs/scheduling/server/grpc_service.go | 24 ++++++ pkg/mcs/scheduling/server/meta/watcher.go | 5 ++ server/grpc_service.go | 53 ++++++++++++- tests/integrations/client/go.mod | 2 +- tests/integrations/client/go.sum | 4 +- tests/integrations/mcs/go.mod | 2 +- tests/integrations/mcs/go.sum | 4 +- .../mcs/scheduling/server_test.go | 60 ++++++++++++++- tests/integrations/tso/go.mod | 2 +- tests/integrations/tso/go.sum | 4 +- tools/pd-api-bench/go.mod | 2 +- tools/pd-api-bench/go.sum | 4 +- 15 files changed, 235 insertions(+), 24 deletions(-) diff --git a/go.mod b/go.mod index 6f6be36d883..bb72c0d5dba 100644 --- a/go.mod +++ b/go.mod @@ -16,21 +16,24 @@ require ( github.com/docker/go-units v0.4.0 github.com/elliotchance/pie/v2 v2.1.0 github.com/gin-contrib/cors v1.4.0 + github.com/gin-contrib/gzip v0.0.1 github.com/gin-contrib/pprof v1.4.0 github.com/gin-gonic/gin v1.8.1 github.com/go-echarts/go-echarts v1.0.0 github.com/gogo/protobuf v1.3.2 github.com/google/btree v1.1.2 + github.com/google/uuid v1.3.0 github.com/gorilla/mux v1.7.4 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/joho/godotenv v1.4.0 + github.com/mailru/easyjson v0.7.6 github.com/mattn/go-shellwords v1.0.12 github.com/mgechev/revive v1.0.2 github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d github.com/pingcap/errcode v0.3.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 + github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 github.com/pingcap/tidb-dashboard v0.0.0-20230911054332-22add1e00511 @@ -39,6 +42,7 @@ require ( github.com/sasha-s/go-deadlock v0.2.0 github.com/shirou/gopsutil/v3 v3.23.3 github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 + github.com/soheilhy/cmux v0.1.4 github.com/spf13/cobra v1.0.0 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.8.2 @@ -48,6 +52,7 @@ require ( github.com/unrolled/render v1.0.1 github.com/urfave/negroni v0.3.0 go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 + go.uber.org/atomic v1.10.0 go.uber.org/goleak v1.1.12 go.uber.org/zap v1.24.0 golang.org/x/exp v0.0.0-20230108222341-4b8118a2686a @@ -92,7 +97,6 @@ require ( github.com/fogleman/gg v1.3.0 // indirect github.com/fsnotify/fsnotify v1.4.9 // indirect github.com/ghodss/yaml v1.0.0 // indirect - github.com/gin-contrib/gzip v0.0.1 github.com/gin-contrib/sse v0.1.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect @@ -112,7 +116,6 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/pprof v0.0.0-20211122183932-1daafda22083 // indirect github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect - github.com/google/uuid v1.3.0 github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect @@ -128,7 +131,6 @@ require ( github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect github.com/leodido/go-urn v1.2.1 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect - github.com/mailru/easyjson v0.7.6 github.com/mattn/go-colorable v0.1.8 // indirect github.com/mattn/go-isatty v0.0.14 // indirect github.com/mattn/go-runewidth v0.0.8 // indirect @@ -158,7 +160,6 @@ require ( github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect github.com/shurcooL/sanitized_anchor_name v1.0.0 // indirect github.com/sirupsen/logrus v1.6.0 // indirect - github.com/soheilhy/cmux v0.1.4 github.com/stretchr/objx v0.5.0 // indirect github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 // indirect github.com/tidwall/gjson v1.9.3 // indirect @@ -173,7 +174,6 @@ require ( github.com/yusufpapurcu/wmi v1.2.2 // indirect // Fix panic in unit test with go >= 1.14, ref: etcd-io/bbolt#201 https://github.com/etcd-io/bbolt/pull/201 go.etcd.io/bbolt v1.3.6 // indirect - go.uber.org/atomic v1.10.0 go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect diff --git a/go.sum b/go.sum index 8640a98a6b3..a007acf869e 100644 --- a/go.sum +++ b/go.sum @@ -439,8 +439,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 h1:Gn8rf2Mb3QDifUQHdtcopqKclc9L11hjhZFYBE65lcw= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 h1:Upb52Po0Ev1lPKQdUT4suRwQ5Z49A7gEmJ0trADKftM= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 19bd891196f..28a04c5640a 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -2,11 +2,13 @@ package server import ( "context" - "errors" "sync/atomic" "time" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/kvproto/pkg/schedulingpb" + "github.com/pingcap/log" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/schedule" @@ -18,6 +20,7 @@ import ( "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" + "go.uber.org/zap" ) // Cluster is used to manage all information for scheduling purpose. @@ -28,6 +31,7 @@ type Cluster struct { ruleManager *placement.RuleManager labelerManager *labeler.RegionLabeler regionStats *statistics.RegionStatistics + labelLevelStats *statistics.LabelStatistics hotStat *statistics.HotStat storage storage.Storage coordinator *schedule.Coordinator @@ -53,6 +57,7 @@ func NewCluster(ctx context.Context, persistConfig *config.PersistConfig, storag persistConfig: persistConfig, hotStat: statistics.NewHotStat(ctx), regionStats: statistics.NewRegionStatistics(basicCluster, persistConfig, ruleManager), + labelLevelStats: statistics.NewLabelStatistics(), storage: storage, clusterID: clusterID, checkMembershipCh: checkMembershipCh, @@ -174,4 +179,74 @@ func (c *Cluster) SwitchAPIServerLeader(new pdpb.PDClient) bool { // UpdateRegionsLabelLevelStats updates the status of the region label level by types. func (c *Cluster) UpdateRegionsLabelLevelStats(regions []*core.RegionInfo) { + for _, region := range regions { + c.labelLevelStats.Observe(region, c.getStoresWithoutLabelLocked(region, core.EngineKey, core.EngineTiFlash), c.persistConfig.GetLocationLabels()) + } +} + +func (c *Cluster) getStoresWithoutLabelLocked(region *core.RegionInfo, key, value string) []*core.StoreInfo { + stores := make([]*core.StoreInfo, 0, len(region.GetPeers())) + for _, p := range region.GetPeers() { + if store := c.GetStore(p.GetStoreId()); store != nil && !core.IsStoreContainLabel(store.GetMeta(), key, value) { + stores = append(stores, store) + } + } + return stores +} + +// HandleStoreHeartbeat updates the store status. +func (c *Cluster) HandleStoreHeartbeat(heartbeat *schedulingpb.StoreHeartbeatRequest) error { + stats := heartbeat.GetStats() + storeID := stats.GetStoreId() + store := c.GetStore(storeID) + if store == nil { + return errors.Errorf("store %v not found", storeID) + } + + nowTime := time.Now() + newStore := store.Clone(core.SetStoreStats(stats), core.SetLastHeartbeatTS(nowTime)) + + if store := c.GetStore(storeID); store != nil { + statistics.UpdateStoreHeartbeatMetrics(store) + } + c.PutStore(newStore) + c.hotStat.Observe(storeID, newStore.GetStoreStats()) + c.hotStat.FilterUnhealthyStore(c) + reportInterval := stats.GetInterval() + interval := reportInterval.GetEndTimestamp() - reportInterval.GetStartTimestamp() + + regions := make(map[uint64]*core.RegionInfo, len(stats.GetPeerStats())) + for _, peerStat := range stats.GetPeerStats() { + regionID := peerStat.GetRegionId() + region := c.GetRegion(regionID) + regions[regionID] = region + if region == nil { + log.Warn("discard hot peer stat for unknown region", + zap.Uint64("region-id", regionID), + zap.Uint64("store-id", storeID)) + continue + } + peer := region.GetStorePeer(storeID) + if peer == nil { + log.Warn("discard hot peer stat for unknown region peer", + zap.Uint64("region-id", regionID), + zap.Uint64("store-id", storeID)) + continue + } + readQueryNum := core.GetReadQueryNum(peerStat.GetQueryStats()) + loads := []float64{ + utils.RegionReadBytes: float64(peerStat.GetReadBytes()), + utils.RegionReadKeys: float64(peerStat.GetReadKeys()), + utils.RegionReadQueryNum: float64(readQueryNum), + utils.RegionWriteBytes: 0, + utils.RegionWriteKeys: 0, + utils.RegionWriteQueryNum: 0, + } + peerInfo := core.NewPeerInfo(peer, loads, interval) + c.hotStat.CheckReadAsync(statistics.NewCheckPeerTask(peerInfo, region)) + } + + // Here we will compare the reported regions with the previous hot peers to decide if it is still hot. + c.hotStat.CheckReadAsync(statistics.NewCollectUnReportedPeerTask(storeID, regions, interval)) + return nil } diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index e75c78eb415..f615e0c37c0 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -15,12 +15,15 @@ package server import ( + "context" "net/http" + "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" + "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -64,8 +67,29 @@ func NewService[T ConfigProvider](svr bs.Server) registry.RegistrableService { } } +// StoreHeartbeat implements gRPC PDServer. +func (s *Service) StoreHeartbeat(ctx context.Context, request *schedulingpb.StoreHeartbeatRequest) (*schedulingpb.StoreHeartbeatResponse, error) { + c := s.GetCluster() + if c == nil { + // TODO: add metrics + log.Info("cluster isn't initialized") + return &schedulingpb.StoreHeartbeatResponse{Header: &schedulingpb.ResponseHeader{ClusterId: s.clusterID}}, nil + } + + if c.GetStore(request.GetStats().GetStoreId()) == nil { + s.metaWatcher.GetStoreWatcher().ForceLoad() + } + + // TODO: add metrics + if err := c.HandleStoreHeartbeat(request); err != nil { + log.Error("handle store heartbeat failed", zap.Error(err)) + } + return &schedulingpb.StoreHeartbeatResponse{Header: &schedulingpb.ResponseHeader{ClusterId: s.clusterID}}, nil +} + // RegisterGRPCService registers the service to gRPC server. func (s *Service) RegisterGRPCService(g *grpc.Server) { + schedulingpb.RegisterSchedulingServer(g, s) } // RegisterRESTHandler registers the service to REST server. diff --git a/pkg/mcs/scheduling/server/meta/watcher.go b/pkg/mcs/scheduling/server/meta/watcher.go index 1e58bbd845f..3dbd0fc8c92 100644 --- a/pkg/mcs/scheduling/server/meta/watcher.go +++ b/pkg/mcs/scheduling/server/meta/watcher.go @@ -115,3 +115,8 @@ func (w *Watcher) Close() { w.cancel() w.wg.Wait() } + +// GetStoreWatcher returns the store watcher. +func (w *Watcher) GetStoreWatcher() *etcdutil.LoopWatcher { + return w.storeWatcher +} diff --git a/server/grpc_service.go b/server/grpc_service.go index 1d20bb22d4d..55b265e32a5 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/core" @@ -76,9 +77,29 @@ var ( // GrpcServer wraps Server to provide grpc service. type GrpcServer struct { *Server + schedulingClient atomic.Value concurrentTSOProxyStreamings atomic.Int32 } +type schedulingClient struct { + client schedulingpb.SchedulingClient + lastPrimary string +} + +func (s *schedulingClient) getClient() schedulingpb.SchedulingClient { + if s == nil { + return nil + } + return s.client +} + +func (s *schedulingClient) getPrimaryAddr() string { + if s == nil { + return "" + } + return s.lastPrimary +} + type request interface { GetHeader() *pdpb.RequestHeader } @@ -978,8 +999,23 @@ func (s *GrpcServer) StoreHeartbeat(ctx context.Context, request *pdpb.StoreHear } s.handleDamagedStore(request.GetStats()) - storeHeartbeatHandleDuration.WithLabelValues(storeAddress, storeLabel).Observe(time.Since(start).Seconds()) + if s.IsAPIServiceMode() { + s.updateSchedulingClient(ctx) + if s.schedulingClient.Load() != nil { + req := &schedulingpb.StoreHeartbeatRequest{ + Header: &schedulingpb.RequestHeader{ + ClusterId: request.GetHeader().GetClusterId(), + SenderId: request.GetHeader().GetSenderId(), + }, + Stats: request.GetStats(), + } + if _, err := s.schedulingClient.Load().(*schedulingClient).getClient().StoreHeartbeat(ctx, req); err != nil { + // reset to let it be updated in the next request + s.schedulingClient.Store(&schedulingClient{}) + } + } + } } if status := request.GetDrAutosyncStatus(); status != nil { @@ -993,6 +1029,21 @@ func (s *GrpcServer) StoreHeartbeat(ctx context.Context, request *pdpb.StoreHear return resp, nil } +func (s *GrpcServer) updateSchedulingClient(ctx context.Context) { + forwardedHost, _ := s.GetServicePrimaryAddr(ctx, utils.SchedulingServiceName) + pre := s.schedulingClient.Load() + if forwardedHost != "" && ((pre == nil) || (pre != nil && forwardedHost != pre.(*schedulingClient).getPrimaryAddr())) { + client, err := s.getDelegateClient(ctx, forwardedHost) + if err != nil { + log.Error("get delegate client failed", zap.Error(err)) + } + s.schedulingClient.Store(&schedulingClient{ + client: schedulingpb.NewSchedulingClient(client), + lastPrimary: forwardedHost, + }) + } +} + // bucketHeartbeatServer wraps PD_ReportBucketsServer to ensure when any error // occurs on SendAndClose() or Recv(), both endpoints will be closed. type bucketHeartbeatServer struct { diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index 70006a085e9..cbb2c2d5f46 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/docker/go-units v0.4.0 github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 + github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index 04eb8e8647f..f18313cb3bb 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -402,8 +402,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 h1:Gn8rf2Mb3QDifUQHdtcopqKclc9L11hjhZFYBE65lcw= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 h1:Upb52Po0Ev1lPKQdUT4suRwQ5Z49A7gEmJ0trADKftM= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index 98c8a420238..50e816cd3ad 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -12,7 +12,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 + github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index 801d9b926cc..8afbfcdc70e 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -407,8 +407,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 h1:Gn8rf2Mb3QDifUQHdtcopqKclc9L11hjhZFYBE65lcw= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 h1:Upb52Po0Ev1lPKQdUT4suRwQ5Z49A7gEmJ0trADKftM= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index 54994bbc34b..f4785a013d9 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -20,9 +20,12 @@ import ( "time" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/suite" mcs "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/testutil" + "github.com/tikv/pd/server" "github.com/tikv/pd/tests" "go.uber.org/goleak" ) @@ -93,8 +96,10 @@ func (suite *serverTestSuite) TestAllocIDAfterLeaderChange() { re.NoError(err) re.NotEqual(uint64(0), id) suite.cluster.ResignLeader() - suite.cluster.WaitLeader() - time.Sleep(200 * time.Millisecond) + leaderName := suite.cluster.WaitLeader() + suite.pdLeader = suite.cluster.GetServer(leaderName) + suite.backendEndpoints = suite.pdLeader.GetAddr() + time.Sleep(time.Second) id1, err := cluster.AllocID() re.NoError(err) re.Greater(id1, id) @@ -126,3 +131,54 @@ func (suite *serverTestSuite) TestPrimaryChange() { return ok && newPrimaryAddr == watchedAddr }) } + +func (suite *serverTestSuite) TestForwardStoreHeartbeat() { + re := suite.Require() + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + + s := &server.GrpcServer{Server: suite.pdLeader.GetServer()} + resp, err := s.PutStore( + context.Background(), &pdpb.PutStoreRequest{ + Header: &pdpb.RequestHeader{ClusterId: suite.pdLeader.GetClusterID()}, + Store: &metapb.Store{ + Id: 1, + Address: "tikv1", + State: metapb.StoreState_Up, + Version: "7.0.0", + }, + }, + ) + re.NoError(err) + re.Empty(resp.GetHeader().GetError()) + + resp1, err := s.StoreHeartbeat( + context.Background(), &pdpb.StoreHeartbeatRequest{ + Header: &pdpb.RequestHeader{ClusterId: suite.pdLeader.GetClusterID()}, + Stats: &pdpb.StoreStats{ + StoreId: 1, + Capacity: 1798985089024, + Available: 1709868695552, + UsedSize: 85150956358, + KeysWritten: 20000, + BytesWritten: 199, + KeysRead: 10000, + BytesRead: 99, + }, + }, + ) + re.NoError(err) + re.Empty(resp1.GetHeader().GetError()) + testutil.Eventually(re, func() bool { + store := tc.GetPrimaryServer().GetCluster().GetStore(1) + return store.GetStoreStats().GetCapacity() == uint64(1798985089024) && + store.GetStoreStats().GetAvailable() == uint64(1709868695552) && + store.GetStoreStats().GetUsedSize() == uint64(85150956358) && + store.GetStoreStats().GetKeysWritten() == uint64(20000) && + store.GetStoreStats().GetBytesWritten() == uint64(199) && + store.GetStoreStats().GetKeysRead() == uint64(10000) && + store.GetStoreStats().GetBytesRead() == uint64(99) + }) +} diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index a9788e01ed2..cea17c73141 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 + github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 github.com/stretchr/testify v1.8.4 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index 891bc42d24f..5ccf67f8b13 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -401,8 +401,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 h1:Gn8rf2Mb3QDifUQHdtcopqKclc9L11hjhZFYBE65lcw= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 h1:Upb52Po0Ev1lPKQdUT4suRwQ5Z49A7gEmJ0trADKftM= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tools/pd-api-bench/go.mod b/tools/pd-api-bench/go.mod index 1cef635e20b..bb3d5be530f 100644 --- a/tools/pd-api-bench/go.mod +++ b/tools/pd-api-bench/go.mod @@ -70,7 +70,7 @@ require ( github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 // indirect - github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 // indirect + github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tools/pd-api-bench/go.sum b/tools/pd-api-bench/go.sum index 2cdd7ca7a38..29d39f504df 100644 --- a/tools/pd-api-bench/go.sum +++ b/tools/pd-api-bench/go.sum @@ -263,8 +263,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974 h1:Gn8rf2Mb3QDifUQHdtcopqKclc9L11hjhZFYBE65lcw= -github.com/pingcap/kvproto v0.0.0-20230905082026-5336fac26974/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96 h1:Upb52Po0Ev1lPKQdUT4suRwQ5Z49A7gEmJ0trADKftM= +github.com/pingcap/kvproto v0.0.0-20230911090708-d603cce32b96/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= From e295e628f485854c3e9f8e092e86e0eae8661e39 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 14 Sep 2023 15:00:11 +0800 Subject: [PATCH 05/10] scheduling: sync schedulers from the API server (#7076) ref tikv/pd#5839 - Sync the schedulers from the API server. - Dynamically reload the scheduler config. Signed-off-by: JmPotato --- pkg/mcs/scheduling/server/cluster.go | 98 +++++++++++++++++++ pkg/mcs/scheduling/server/config/config.go | 35 ++++++- pkg/mcs/scheduling/server/config/watcher.go | 39 +++++++- pkg/mcs/scheduling/server/server.go | 2 + pkg/schedule/coordinator.go | 54 ++++++---- pkg/schedule/schedulers/balance_leader.go | 21 +++- pkg/schedule/schedulers/balance_witness.go | 21 +++- pkg/schedule/schedulers/base_scheduler.go | 5 + pkg/schedule/schedulers/evict_leader.go | 20 +++- pkg/schedule/schedulers/evict_slow_store.go | 2 +- .../schedulers/evict_slow_store_test.go | 2 +- pkg/schedule/schedulers/evict_slow_trend.go | 2 +- .../schedulers/evict_slow_trend_test.go | 2 +- pkg/schedule/schedulers/grant_hot_region.go | 2 +- pkg/schedule/schedulers/grant_leader.go | 20 +++- pkg/schedule/schedulers/hot_region_config.go | 2 +- pkg/schedule/schedulers/hot_region_test.go | 4 +- pkg/schedule/schedulers/scatter_range.go | 15 ++- pkg/schedule/schedulers/scheduler.go | 4 +- .../schedulers/scheduler_controller.go | 12 ++- pkg/schedule/schedulers/shuffle_region.go | 19 ++++ .../schedulers/shuffle_region_config.go | 2 +- pkg/schedule/schedulers/split_bucket.go | 15 ++- .../schedulers/transfer_witness_leader.go | 18 ++-- pkg/storage/endpoint/config.go | 32 +++--- pkg/storage/endpoint/key_path.go | 28 +++--- plugin/scheduler_example/evict_leader.go | 2 +- server/cluster/cluster_test.go | 8 +- server/server.go | 2 +- .../mcs/scheduling/config_test.go | 32 +++--- .../mcs/scheduling/server_test.go | 69 +++++++++++++ 31 files changed, 483 insertions(+), 106 deletions(-) diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 28a04c5640a..61c85ba6fbd 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -10,16 +10,20 @@ import ( "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/core" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/schedule" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/placement" + "github.com/tikv/pd/pkg/schedule/schedulers" + "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage" + "github.com/tikv/pd/pkg/utils/logutil" "go.uber.org/zap" ) @@ -175,6 +179,100 @@ func (c *Cluster) SwitchAPIServerLeader(new pdpb.PDClient) bool { return c.apiServerLeader.CompareAndSwap(old, new) } +// UpdateScheduler listens on the schedulers updating notifier and manage the scheduler creation and deletion. +func (c *Cluster) UpdateScheduler() { + defer logutil.LogPanic() + + // Make sure the coordinator has initialized all the existing schedulers. + c.waitSchedulersInitialized() + // Establish a notifier to listen the schedulers updating. + notifier := make(chan struct{}, 1) + // Make sure the check will be triggered once later. + notifier <- struct{}{} + c.persistConfig.SetSchedulersUpdatingNotifier(notifier) + for { + select { + case <-c.ctx.Done(): + log.Info("cluster is closing, stop listening the schedulers updating notifier") + return + case <-notifier: + } + + log.Info("schedulers updating notifier is triggered, try to update the scheduler") + var ( + schedulersController = c.coordinator.GetSchedulersController() + latestSchedulersConfig = c.persistConfig.GetScheduleConfig().Schedulers + ) + // Create the newly added schedulers. + for _, scheduler := range latestSchedulersConfig { + s, err := schedulers.CreateScheduler( + scheduler.Type, + c.coordinator.GetOperatorController(), + c.storage, + schedulers.ConfigSliceDecoder(scheduler.Type, scheduler.Args), + schedulersController.RemoveScheduler, + ) + if err != nil { + log.Error("failed to create scheduler", + zap.String("scheduler-type", scheduler.Type), + zap.Strings("scheduler-args", scheduler.Args), + errs.ZapError(err)) + continue + } + name := s.GetName() + if existed, _ := schedulersController.IsSchedulerExisted(name); existed { + log.Info("scheduler has already existed, skip adding it", + zap.String("scheduler-name", name), + zap.Strings("scheduler-args", scheduler.Args)) + continue + } + if err := schedulersController.AddScheduler(s, scheduler.Args...); err != nil { + log.Error("failed to add scheduler", + zap.String("scheduler-name", name), + zap.Strings("scheduler-args", scheduler.Args), + errs.ZapError(err)) + continue + } + log.Info("add scheduler successfully", + zap.String("scheduler-name", name), + zap.Strings("scheduler-args", scheduler.Args)) + } + // Remove the deleted schedulers. + for _, name := range schedulersController.GetSchedulerNames() { + scheduler := schedulersController.GetScheduler(name) + if slice.AnyOf(latestSchedulersConfig, func(i int) bool { + return latestSchedulersConfig[i].Type == scheduler.GetType() + }) { + continue + } + if err := schedulersController.RemoveScheduler(name); err != nil { + log.Error("failed to remove scheduler", + zap.String("scheduler-name", name), + errs.ZapError(err)) + continue + } + log.Info("remove scheduler successfully", + zap.String("scheduler-name", name)) + } + } +} + +func (c *Cluster) waitSchedulersInitialized() { + ticker := time.NewTicker(time.Millisecond * 100) + defer ticker.Stop() + for { + if c.coordinator.AreSchedulersInitialized() { + return + } + select { + case <-c.ctx.Done(): + log.Info("cluster is closing, stop waiting the schedulers initialization") + return + case <-ticker.C: + } + } +} + // TODO: implement the following methods // UpdateRegionsLabelLevelStats updates the status of the region label level by types. diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index 5b80612744b..82c15632b3d 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -203,6 +203,9 @@ type PersistConfig struct { schedule atomic.Value replication atomic.Value storeConfig atomic.Value + // schedulersUpdatingNotifier is used to notify that the schedulers have been updated. + // Store as `chan<- struct{}`. + schedulersUpdatingNotifier atomic.Value } // NewPersistConfig creates a new PersistConfig instance. @@ -217,6 +220,19 @@ func NewPersistConfig(cfg *Config) *PersistConfig { return o } +// SetSchedulersUpdatingNotifier sets the schedulers updating notifier. +func (o *PersistConfig) SetSchedulersUpdatingNotifier(notifier chan<- struct{}) { + o.schedulersUpdatingNotifier.Store(notifier) +} + +func (o *PersistConfig) getSchedulersUpdatingNotifier() chan<- struct{} { + v := o.schedulersUpdatingNotifier.Load() + if v == nil { + return nil + } + return v.(chan<- struct{}) +} + // GetClusterVersion returns the cluster version. func (o *PersistConfig) GetClusterVersion() *semver.Version { return (*semver.Version)(atomic.LoadPointer(&o.clusterVersion)) @@ -232,12 +248,19 @@ func (o *PersistConfig) GetScheduleConfig() *sc.ScheduleConfig { return o.schedule.Load().(*sc.ScheduleConfig) } -// SetScheduleConfig sets the scheduling configuration. +// SetScheduleConfig sets the scheduling configuration dynamically. func (o *PersistConfig) SetScheduleConfig(cfg *sc.ScheduleConfig) { + old := o.GetScheduleConfig() o.schedule.Store(cfg) + // The coordinator is not aware of the underlying scheduler config changes, however, it + // should react on the scheduler number changes to handle the add/remove scheduler events. + if notifier := o.getSchedulersUpdatingNotifier(); notifier != nil && + len(old.Schedulers) != len(cfg.Schedulers) { + notifier <- struct{}{} + } } -// AdjustScheduleCfg adjusts the schedule config. +// AdjustScheduleCfg adjusts the schedule config during the initialization. func (o *PersistConfig) AdjustScheduleCfg(scheduleCfg *sc.ScheduleConfig) { // In case we add new default schedulers. for _, ps := range sc.DefaultSchedulers { @@ -616,8 +639,13 @@ func (o *PersistConfig) IsRaftKV2() bool { // TODO: implement the following methods // AddSchedulerCfg adds the scheduler configurations. +// This method is a no-op since we only use configurations derived from one-way synchronization from API server now. func (o *PersistConfig) AddSchedulerCfg(string, []string) {} +// RemoveSchedulerCfg removes the scheduler configurations. +// This method is a no-op since we only use configurations derived from one-way synchronization from API server now. +func (o *PersistConfig) RemoveSchedulerCfg(tp string) {} + // CheckLabelProperty checks if the label property is satisfied. func (o *PersistConfig) CheckLabelProperty(typ string, labels []*metapb.StoreLabel) bool { return false @@ -633,6 +661,3 @@ func (o *PersistConfig) IsTraceRegionFlow() bool { func (o *PersistConfig) Persist(storage endpoint.ConfigStorage) error { return nil } - -// RemoveSchedulerCfg removes the scheduler configurations. -func (o *PersistConfig) RemoveSchedulerCfg(tp string) {} diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index d65f1a6b553..b413e243c2b 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -19,10 +19,12 @@ import ( "encoding/json" "strings" "sync" + "sync/atomic" "github.com/coreos/go-semver/semver" "github.com/pingcap/log" sc "github.com/tikv/pd/pkg/schedule/config" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" @@ -50,10 +52,14 @@ type Watcher struct { configWatcher *etcdutil.LoopWatcher schedulerConfigWatcher *etcdutil.LoopWatcher + // Some data, like the global schedule config, should be loaded into `PersistConfig`. *PersistConfig // Some data, like the scheduler configs, should be loaded into the storage // to make sure the coordinator could access them correctly. storage storage.Storage + // schedulersController is used to trigger the scheduler's config reloading. + // Store as `*schedulers.Controller`. + schedulersController atomic.Value } type persistedConfig struct { @@ -92,6 +98,19 @@ func NewWatcher( return cw, nil } +// SetSchedulersController sets the schedulers controller. +func (cw *Watcher) SetSchedulersController(sc *schedulers.Controller) { + cw.schedulersController.Store(sc) +} + +func (cw *Watcher) getSchedulersController() *schedulers.Controller { + sc := cw.schedulersController.Load() + if sc == nil { + return nil + } + return sc.(*schedulers.Controller) +} + func (cw *Watcher) initializeConfigWatcher() error { putFn := func(kv *mvccpb.KeyValue) error { cfg := &persistedConfig{} @@ -126,13 +145,23 @@ func (cw *Watcher) initializeConfigWatcher() error { func (cw *Watcher) initializeSchedulerConfigWatcher() error { prefixToTrim := cw.schedulerConfigPathPrefix + "/" putFn := func(kv *mvccpb.KeyValue) error { - return cw.storage.SaveScheduleConfig( - strings.TrimPrefix(string(kv.Key), prefixToTrim), - kv.Value, - ) + name := strings.TrimPrefix(string(kv.Key), prefixToTrim) + err := cw.storage.SaveSchedulerConfig(name, kv.Value) + if err != nil { + log.Warn("failed to save scheduler config", + zap.String("event-kv-key", string(kv.Key)), + zap.String("trimmed-key", name), + zap.Error(err)) + return err + } + // Ensure the scheduler config could be updated as soon as possible. + if sc := cw.getSchedulersController(); sc != nil { + return sc.ReloadSchedulerConfig(name) + } + return nil } deleteFn := func(kv *mvccpb.KeyValue) error { - return cw.storage.RemoveScheduleConfig( + return cw.storage.RemoveSchedulerConfig( strings.TrimPrefix(string(kv.Key), prefixToTrim), ) } diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 8a74abf4b5e..330085e3b82 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -442,6 +442,8 @@ func (s *Server) startCluster(context.Context) error { if err != nil { return err } + s.configWatcher.SetSchedulersController(s.cluster.GetCoordinator().GetSchedulersController()) + go s.cluster.UpdateScheduler() go s.GetCoordinator().RunUntilStop() return nil } diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index ab0500d0445..8cd5567b75c 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -69,9 +69,12 @@ var ( type Coordinator struct { syncutil.RWMutex - wg sync.WaitGroup - ctx context.Context - cancel context.CancelFunc + wg sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + schedulersInitialized bool + cluster sche.ClusterInformer prepareChecker *prepareChecker checkers *checker.Controller @@ -91,19 +94,34 @@ func NewCoordinator(ctx context.Context, cluster sche.ClusterInformer, hbStreams schedulers := schedulers.NewController(ctx, cluster, cluster.GetStorage(), opController) checkers := checker.NewController(ctx, cluster, cluster.GetCheckerConfig(), cluster.GetRuleManager(), cluster.GetRegionLabeler(), opController) return &Coordinator{ - ctx: ctx, - cancel: cancel, - cluster: cluster, - prepareChecker: newPrepareChecker(), - checkers: checkers, - regionScatterer: scatter.NewRegionScatterer(ctx, cluster, opController, checkers.AddSuspectRegions), - regionSplitter: splitter.NewRegionSplitter(cluster, splitter.NewSplitRegionsHandler(cluster, opController), checkers.AddSuspectRegions), - schedulers: schedulers, - opController: opController, - hbStreams: hbStreams, - pluginInterface: NewPluginInterface(), - diagnosticManager: diagnostic.NewManager(schedulers, cluster.GetSchedulerConfig()), - } + ctx: ctx, + cancel: cancel, + schedulersInitialized: false, + cluster: cluster, + prepareChecker: newPrepareChecker(), + checkers: checkers, + regionScatterer: scatter.NewRegionScatterer(ctx, cluster, opController, checkers.AddSuspectRegions), + regionSplitter: splitter.NewRegionSplitter(cluster, splitter.NewSplitRegionsHandler(cluster, opController), checkers.AddSuspectRegions), + schedulers: schedulers, + opController: opController, + hbStreams: hbStreams, + pluginInterface: NewPluginInterface(), + diagnosticManager: diagnostic.NewManager(schedulers, cluster.GetSchedulerConfig()), + } +} + +// markSchedulersInitialized marks the scheduler initialization is finished. +func (c *Coordinator) markSchedulersInitialized() { + c.Lock() + defer c.Unlock() + c.schedulersInitialized = true +} + +// AreSchedulersInitialized returns whether the schedulers have been initialized. +func (c *Coordinator) AreSchedulersInitialized() bool { + c.RLock() + defer c.RUnlock() + return c.schedulersInitialized } // GetWaitingRegions returns the regions in the waiting list. @@ -399,7 +417,7 @@ func (c *Coordinator) InitSchedulers(needRun bool) { err error ) for i := 0; i < maxLoadConfigRetries; i++ { - scheduleNames, configs, err = c.cluster.GetStorage().LoadAllScheduleConfig() + scheduleNames, configs, err = c.cluster.GetStorage().LoadAllSchedulerConfigs() select { case <-c.ctx.Done(): log.Info("init schedulers has been stopped") @@ -485,6 +503,8 @@ func (c *Coordinator) InitSchedulers(needRun bool) { if err := c.cluster.GetSchedulerConfig().Persist(c.cluster.GetStorage()); err != nil { log.Error("cannot persist schedule config", errs.ZapError(err)) } + + c.markSchedulersInitialized() } // LoadPlugin load user plugin diff --git a/pkg/schedule/schedulers/balance_leader.go b/pkg/schedule/schedulers/balance_leader.go index 70f1cf5cb53..e5516317f46 100644 --- a/pkg/schedule/schedulers/balance_leader.go +++ b/pkg/schedule/schedulers/balance_leader.go @@ -124,7 +124,7 @@ func (conf *balanceLeaderSchedulerConfig) persistLocked() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(BalanceLeaderName, data) + return conf.storage.SaveSchedulerConfig(BalanceLeaderName, data) } type balanceLeaderHandler struct { @@ -215,6 +215,25 @@ func (l *balanceLeaderScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(l.conf) } +func (l *balanceLeaderScheduler) ReloadConfig() error { + l.conf.mu.Lock() + defer l.conf.mu.Unlock() + cfgData, err := l.conf.storage.LoadSchedulerConfig(l.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + newCfg := &balanceLeaderSchedulerConfig{} + if err = DecodeConfig([]byte(cfgData), newCfg); err != nil { + return err + } + l.conf.Ranges = newCfg.Ranges + l.conf.Batch = newCfg.Batch + return nil +} + func (l *balanceLeaderScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { allowed := l.OpController.OperatorCount(operator.OpLeader) < cluster.GetSchedulerConfig().GetLeaderScheduleLimit() if !allowed { diff --git a/pkg/schedule/schedulers/balance_witness.go b/pkg/schedule/schedulers/balance_witness.go index b60c6d81fa0..9bd8a592ba1 100644 --- a/pkg/schedule/schedulers/balance_witness.go +++ b/pkg/schedule/schedulers/balance_witness.go @@ -110,7 +110,7 @@ func (conf *balanceWitnessSchedulerConfig) persistLocked() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(BalanceWitnessName, data) + return conf.storage.SaveSchedulerConfig(BalanceWitnessName, data) } type balanceWitnessHandler struct { @@ -210,6 +210,25 @@ func (b *balanceWitnessScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(b.conf) } +func (b *balanceWitnessScheduler) ReloadConfig() error { + b.conf.mu.Lock() + defer b.conf.mu.Unlock() + cfgData, err := b.conf.storage.LoadSchedulerConfig(b.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + newCfg := &balanceWitnessSchedulerConfig{} + if err = DecodeConfig([]byte(cfgData), newCfg); err != nil { + return err + } + b.conf.Ranges = newCfg.Ranges + b.conf.Batch = newCfg.Batch + return nil +} + func (b *balanceWitnessScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { allowed := b.OpController.OperatorCount(operator.OpWitness) < cluster.GetSchedulerConfig().GetWitnessScheduleLimit() if !allowed { diff --git a/pkg/schedule/schedulers/base_scheduler.go b/pkg/schedule/schedulers/base_scheduler.go index 8da0f13626b..6e712c18fe3 100644 --- a/pkg/schedule/schedulers/base_scheduler.go +++ b/pkg/schedule/schedulers/base_scheduler.go @@ -82,6 +82,11 @@ func (s *BaseScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(nil) } +// ReloadConfig reloads the config from the storage. +// By default, the scheduler does not need to reload the config +// if it doesn't support the dynamic configuration. +func (s *BaseScheduler) ReloadConfig() error { return nil } + // GetNextInterval return the next interval for the scheduler func (s *BaseScheduler) GetNextInterval(interval time.Duration) time.Duration { return intervalGrow(interval, MaxScheduleInterval, exponentialGrowth) diff --git a/pkg/schedule/schedulers/evict_leader.go b/pkg/schedule/schedulers/evict_leader.go index b307bf5fb73..1989c42ba6f 100644 --- a/pkg/schedule/schedulers/evict_leader.go +++ b/pkg/schedule/schedulers/evict_leader.go @@ -115,7 +115,7 @@ func (conf *evictLeaderSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *evictLeaderSchedulerConfig) getSchedulerName() string { @@ -204,6 +204,24 @@ func (s *evictLeaderScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(s.conf) } +func (s *evictLeaderScheduler) ReloadConfig() error { + s.conf.mu.Lock() + defer s.conf.mu.Unlock() + cfgData, err := s.conf.storage.LoadSchedulerConfig(s.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + newCfg := &evictLeaderSchedulerConfig{} + if err = DecodeConfig([]byte(cfgData), newCfg); err != nil { + return err + } + s.conf.StoreIDWithRanges = newCfg.StoreIDWithRanges + return nil +} + func (s *evictLeaderScheduler) Prepare(cluster sche.SchedulerCluster) error { s.conf.mu.RLock() defer s.conf.mu.RUnlock() diff --git a/pkg/schedule/schedulers/evict_slow_store.go b/pkg/schedule/schedulers/evict_slow_store.go index ead92378398..a6665c3e5e7 100644 --- a/pkg/schedule/schedulers/evict_slow_store.go +++ b/pkg/schedule/schedulers/evict_slow_store.go @@ -53,7 +53,7 @@ func (conf *evictSlowStoreSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *evictSlowStoreSchedulerConfig) getSchedulerName() string { diff --git a/pkg/schedule/schedulers/evict_slow_store_test.go b/pkg/schedule/schedulers/evict_slow_store_test.go index 58a44118048..0b0c1d9ad39 100644 --- a/pkg/schedule/schedulers/evict_slow_store_test.go +++ b/pkg/schedule/schedulers/evict_slow_store_test.go @@ -99,7 +99,7 @@ func (suite *evictSlowStoreTestSuite) TestEvictSlowStore() { suite.Zero(es2.conf.evictStore()) // check the value from storage. - sches, vs, err := es2.conf.storage.LoadAllScheduleConfig() + sches, vs, err := es2.conf.storage.LoadAllSchedulerConfigs() suite.NoError(err) valueStr := "" for id, sche := range sches { diff --git a/pkg/schedule/schedulers/evict_slow_trend.go b/pkg/schedule/schedulers/evict_slow_trend.go index 8737e3b8619..38a6141cd26 100644 --- a/pkg/schedule/schedulers/evict_slow_trend.go +++ b/pkg/schedule/schedulers/evict_slow_trend.go @@ -68,7 +68,7 @@ func (conf *evictSlowTrendSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *evictSlowTrendSchedulerConfig) getSchedulerName() string { diff --git a/pkg/schedule/schedulers/evict_slow_trend_test.go b/pkg/schedule/schedulers/evict_slow_trend_test.go index 9320c3ad422..2ff86524bdc 100644 --- a/pkg/schedule/schedulers/evict_slow_trend_test.go +++ b/pkg/schedule/schedulers/evict_slow_trend_test.go @@ -181,7 +181,7 @@ func (suite *evictSlowTrendTestSuite) TestEvictSlowTrend() { suite.Zero(es2.conf.evictedStore()) // check the value from storage. - sches, vs, err := es2.conf.storage.LoadAllScheduleConfig() + sches, vs, err := es2.conf.storage.LoadAllSchedulerConfigs() suite.NoError(err) valueStr := "" for id, sche := range sches { diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 4c8051de677..5a68da069b8 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -105,7 +105,7 @@ func (conf *grantHotRegionSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *grantHotRegionSchedulerConfig) getSchedulerName() string { diff --git a/pkg/schedule/schedulers/grant_leader.go b/pkg/schedule/schedulers/grant_leader.go index 08856d101cc..7d1ff2f616c 100644 --- a/pkg/schedule/schedulers/grant_leader.go +++ b/pkg/schedule/schedulers/grant_leader.go @@ -95,7 +95,7 @@ func (conf *grantLeaderSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *grantLeaderSchedulerConfig) getSchedulerName() string { @@ -178,6 +178,24 @@ func (s *grantLeaderScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(s.conf) } +func (s *grantLeaderScheduler) ReloadConfig() error { + s.conf.mu.Lock() + defer s.conf.mu.Unlock() + cfgData, err := s.conf.storage.LoadSchedulerConfig(s.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + newCfg := &grantLeaderSchedulerConfig{} + if err = DecodeConfig([]byte(cfgData), newCfg); err != nil { + return err + } + s.conf.StoreIDWithRanges = newCfg.StoreIDWithRanges + return nil +} + func (s *grantLeaderScheduler) Prepare(cluster sche.SchedulerCluster) error { s.conf.mu.RLock() defer s.conf.mu.RUnlock() diff --git a/pkg/schedule/schedulers/hot_region_config.go b/pkg/schedule/schedulers/hot_region_config.go index e0fd47c6447..2ff78748f02 100644 --- a/pkg/schedule/schedulers/hot_region_config.go +++ b/pkg/schedule/schedulers/hot_region_config.go @@ -445,7 +445,7 @@ func (conf *hotRegionSchedulerConfig) persistLocked() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(HotRegionName, data) + return conf.storage.SaveSchedulerConfig(HotRegionName, data) } func (conf *hotRegionSchedulerConfig) checkQuerySupport(cluster sche.SchedulerCluster) bool { diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index b29898cd9b9..d8f9bbc532c 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -2456,7 +2456,7 @@ func TestCompatibilityConfig(t *testing.T) { "dst-tolerance-ratio": 1.05, }) re.NoError(err) - err = storage.SaveScheduleConfig(HotRegionName, data) + err = storage.SaveSchedulerConfig(HotRegionName, data) re.NoError(err) hb, err = CreateScheduler(HotRegionType, oc, storage, ConfigJSONDecoder(data)) re.NoError(err) @@ -2472,7 +2472,7 @@ func TestCompatibilityConfig(t *testing.T) { cfg.WriteLeaderPriorities = []string{"query", "key"} data, err = EncodeConfig(cfg) re.NoError(err) - err = storage.SaveScheduleConfig(HotRegionName, data) + err = storage.SaveSchedulerConfig(HotRegionName, data) re.NoError(err) hb, err = CreateScheduler(HotRegionType, oc, storage, ConfigJSONDecoder(data)) re.NoError(err) diff --git a/pkg/schedule/schedulers/scatter_range.go b/pkg/schedule/schedulers/scatter_range.go index c0e8e900795..e301b4c6e76 100644 --- a/pkg/schedule/schedulers/scatter_range.go +++ b/pkg/schedule/schedulers/scatter_range.go @@ -87,7 +87,7 @@ func (conf *scatterRangeSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *scatterRangeSchedulerConfig) GetRangeName() string { @@ -166,6 +166,19 @@ func (l *scatterRangeScheduler) EncodeConfig() ([]byte, error) { return EncodeConfig(l.config) } +func (l *scatterRangeScheduler) ReloadConfig() error { + l.config.mu.Lock() + defer l.config.mu.Unlock() + cfgData, err := l.config.storage.LoadSchedulerConfig(l.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + return DecodeConfig([]byte(cfgData), l.config) +} + func (l *scatterRangeScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { return l.allowBalanceLeader(cluster) || l.allowBalanceRegion(cluster) } diff --git a/pkg/schedule/schedulers/scheduler.go b/pkg/schedule/schedulers/scheduler.go index 909acb3494c..1c624dcd916 100644 --- a/pkg/schedule/schedulers/scheduler.go +++ b/pkg/schedule/schedulers/scheduler.go @@ -38,6 +38,8 @@ type Scheduler interface { // GetType should in accordance with the name passing to RegisterScheduler() GetType() string EncodeConfig() ([]byte, error) + // ReloadConfig reloads the config from the storage. + ReloadConfig() error GetMinInterval() time.Duration GetNextInterval(interval time.Duration) time.Duration Prepare(cluster sche.SchedulerCluster) error @@ -130,7 +132,7 @@ func CreateScheduler(typ string, oc *operator.Controller, storage endpoint.Confi if err != nil { return nil, err } - err = storage.SaveScheduleConfig(s.GetName(), data) + err = storage.SaveSchedulerConfig(s.GetName(), data) return s, err } diff --git a/pkg/schedule/schedulers/scheduler_controller.go b/pkg/schedule/schedulers/scheduler_controller.go index c8f07f56678..4d72699b0fe 100644 --- a/pkg/schedule/schedulers/scheduler_controller.go +++ b/pkg/schedule/schedulers/scheduler_controller.go @@ -160,7 +160,7 @@ func (c *Controller) RemoveSchedulerHandler(name string) error { return err } - if err := c.storage.RemoveScheduleConfig(name); err != nil { + if err := c.storage.RemoveSchedulerConfig(name); err != nil { log.Error("can not remove the scheduler config", errs.ZapError(err)) return err } @@ -210,7 +210,7 @@ func (c *Controller) RemoveScheduler(name string) error { return err } - if err := c.storage.RemoveScheduleConfig(name); err != nil { + if err := c.storage.RemoveSchedulerConfig(name); err != nil { log.Error("can not remove the scheduler config", errs.ZapError(err)) return err } @@ -253,6 +253,14 @@ func (c *Controller) PauseOrResumeScheduler(name string, t int64) error { return err } +// ReloadSchedulerConfig reloads a scheduler's config if it exists. +func (c *Controller) ReloadSchedulerConfig(name string) error { + if exist, _ := c.IsSchedulerExisted(name); !exist { + return nil + } + return c.GetScheduler(name).ReloadConfig() +} + // IsSchedulerAllowed returns whether a scheduler is allowed to schedule, a scheduler is not allowed to schedule if it is paused or blocked by unsafe recovery. func (c *Controller) IsSchedulerAllowed(name string) (bool, error) { c.RLock() diff --git a/pkg/schedule/schedulers/shuffle_region.go b/pkg/schedule/schedulers/shuffle_region.go index 08570fe9f20..f1d35e80925 100644 --- a/pkg/schedule/schedulers/shuffle_region.go +++ b/pkg/schedule/schedulers/shuffle_region.go @@ -80,6 +80,25 @@ func (s *shuffleRegionScheduler) EncodeConfig() ([]byte, error) { return s.conf.EncodeConfig() } +func (s *shuffleRegionScheduler) ReloadConfig() error { + s.conf.Lock() + defer s.conf.Unlock() + cfgData, err := s.conf.storage.LoadSchedulerConfig(s.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + newCfg := &shuffleRegionSchedulerConfig{} + if err := DecodeConfig([]byte(cfgData), newCfg); err != nil { + return err + } + s.conf.Roles = newCfg.Roles + s.conf.Ranges = newCfg.Ranges + return nil +} + func (s *shuffleRegionScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { allowed := s.OpController.OperatorCount(operator.OpRegion) < cluster.GetSchedulerConfig().GetRegionScheduleLimit() if !allowed { diff --git a/pkg/schedule/schedulers/shuffle_region_config.go b/pkg/schedule/schedulers/shuffle_region_config.go index b16f20af8c9..f503a6f67c7 100644 --- a/pkg/schedule/schedulers/shuffle_region_config.go +++ b/pkg/schedule/schedulers/shuffle_region_config.go @@ -109,5 +109,5 @@ func (conf *shuffleRegionSchedulerConfig) persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(ShuffleRegionName, data) + return conf.storage.SaveSchedulerConfig(ShuffleRegionName, data) } diff --git a/pkg/schedule/schedulers/split_bucket.go b/pkg/schedule/schedulers/split_bucket.go index b5dbec7ecd7..5e75bded9b4 100644 --- a/pkg/schedule/schedulers/split_bucket.go +++ b/pkg/schedule/schedulers/split_bucket.go @@ -84,7 +84,7 @@ func (conf *splitBucketSchedulerConfig) persistLocked() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(SplitBucketName, data) + return conf.storage.SaveSchedulerConfig(SplitBucketName, data) } type splitBucketScheduler struct { @@ -172,6 +172,19 @@ func (s *splitBucketScheduler) GetType() string { return SplitBucketType } +func (s *splitBucketScheduler) ReloadConfig() error { + s.conf.mu.Lock() + defer s.conf.mu.Unlock() + cfgData, err := s.conf.storage.LoadSchedulerConfig(s.GetName()) + if err != nil { + return err + } + if len(cfgData) == 0 { + return nil + } + return DecodeConfig([]byte(cfgData), s.conf) +} + // ServerHTTP implement Http server. func (s *splitBucketScheduler) ServeHTTP(w http.ResponseWriter, r *http.Request) { s.handler.ServeHTTP(w, r) diff --git a/pkg/schedule/schedulers/transfer_witness_leader.go b/pkg/schedule/schedulers/transfer_witness_leader.go index d31bb7c1c23..2586065ea80 100644 --- a/pkg/schedule/schedulers/transfer_witness_leader.go +++ b/pkg/schedule/schedulers/transfer_witness_leader.go @@ -46,37 +46,37 @@ var ( transferWitnessLeaderNoTargetStoreCounter = schedulerCounter.WithLabelValues(TransferWitnessLeaderName, "no-target-store") ) -type trasferWitnessLeaderScheduler struct { +type transferWitnessLeaderScheduler struct { *BaseScheduler regions chan *core.RegionInfo } // newTransferWitnessLeaderScheduler creates an admin scheduler that transfers witness leader of a region. func newTransferWitnessLeaderScheduler(opController *operator.Controller) Scheduler { - return &trasferWitnessLeaderScheduler{ + return &transferWitnessLeaderScheduler{ BaseScheduler: NewBaseScheduler(opController), regions: make(chan *core.RegionInfo, transferWitnessLeaderRecvMaxRegionSize), } } -func (s *trasferWitnessLeaderScheduler) GetName() string { +func (s *transferWitnessLeaderScheduler) GetName() string { return TransferWitnessLeaderName } -func (s *trasferWitnessLeaderScheduler) GetType() string { +func (s *transferWitnessLeaderScheduler) GetType() string { return TransferWitnessLeaderType } -func (s *trasferWitnessLeaderScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { +func (s *transferWitnessLeaderScheduler) IsScheduleAllowed(cluster sche.SchedulerCluster) bool { return true } -func (s *trasferWitnessLeaderScheduler) Schedule(cluster sche.SchedulerCluster, dryRun bool) ([]*operator.Operator, []plan.Plan) { +func (s *transferWitnessLeaderScheduler) Schedule(cluster sche.SchedulerCluster, dryRun bool) ([]*operator.Operator, []plan.Plan) { transferWitnessLeaderCounter.Inc() return s.scheduleTransferWitnessLeaderBatch(s.GetName(), s.GetType(), cluster, transferWitnessLeaderBatchSize), nil } -func (s *trasferWitnessLeaderScheduler) scheduleTransferWitnessLeaderBatch(name, typ string, cluster sche.SchedulerCluster, batchSize int) []*operator.Operator { +func (s *transferWitnessLeaderScheduler) scheduleTransferWitnessLeaderBatch(name, typ string, cluster sche.SchedulerCluster, batchSize int) []*operator.Operator { var ops []*operator.Operator batchLoop: for i := 0; i < batchSize; i++ { @@ -99,7 +99,7 @@ batchLoop: return ops } -func (s *trasferWitnessLeaderScheduler) scheduleTransferWitnessLeader(name, typ string, cluster sche.SchedulerCluster, region *core.RegionInfo) (*operator.Operator, error) { +func (s *transferWitnessLeaderScheduler) scheduleTransferWitnessLeader(name, typ string, cluster sche.SchedulerCluster, region *core.RegionInfo) (*operator.Operator, error) { var filters []filter.Filter unhealthyPeerStores := make(map[uint64]struct{}) for _, peer := range region.GetDownPeers() { @@ -127,5 +127,5 @@ func (s *trasferWitnessLeaderScheduler) scheduleTransferWitnessLeader(name, typ // RecvRegionInfo receives a checked region from coordinator func RecvRegionInfo(s Scheduler) chan<- *core.RegionInfo { - return s.(*trasferWitnessLeaderScheduler).regions + return s.(*transferWitnessLeaderScheduler).regions } diff --git a/pkg/storage/endpoint/config.go b/pkg/storage/endpoint/config.go index 9104e218f98..db5565a4b90 100644 --- a/pkg/storage/endpoint/config.go +++ b/pkg/storage/endpoint/config.go @@ -24,11 +24,14 @@ import ( // ConfigStorage defines the storage operations on the config. type ConfigStorage interface { + // Persisted config will be stored in the storage. LoadConfig(cfg interface{}) (bool, error) SaveConfig(cfg interface{}) error - LoadAllScheduleConfig() ([]string, []string, error) - SaveScheduleConfig(scheduleName string, data []byte) error - RemoveScheduleConfig(scheduleName string) error + // Each scheduler has its own customized config, so we need to store them separately. + LoadAllSchedulerConfigs() ([]string, []string, error) + LoadSchedulerConfig(schedulerName string) (string, error) + SaveSchedulerConfig(schedulerName string, data []byte) error + RemoveSchedulerConfig(schedulerName string) error } var _ ConfigStorage = (*StorageEndpoint)(nil) @@ -55,9 +58,9 @@ func (se *StorageEndpoint) SaveConfig(cfg interface{}) error { return se.Save(configPath, string(value)) } -// LoadAllScheduleConfig loads all schedulers' config. -func (se *StorageEndpoint) LoadAllScheduleConfig() ([]string, []string, error) { - prefix := customScheduleConfigPath + "/" +// LoadAllSchedulerConfigs loads all schedulers' config. +func (se *StorageEndpoint) LoadAllSchedulerConfigs() ([]string, []string, error) { + prefix := customSchedulerConfigPath + "/" keys, values, err := se.LoadRange(prefix, clientv3.GetPrefixRangeEnd(prefix), 1000) for i, key := range keys { keys[i] = strings.TrimPrefix(key, prefix) @@ -65,12 +68,17 @@ func (se *StorageEndpoint) LoadAllScheduleConfig() ([]string, []string, error) { return keys, values, err } -// SaveScheduleConfig saves the config of scheduler. -func (se *StorageEndpoint) SaveScheduleConfig(scheduleName string, data []byte) error { - return se.Save(scheduleConfigPath(scheduleName), string(data)) +// LoadSchedulerConfig loads the config of the given scheduler. +func (se *StorageEndpoint) LoadSchedulerConfig(schedulerName string) (string, error) { + return se.Load(schedulerConfigPath(schedulerName)) } -// RemoveScheduleConfig removes the config of scheduler. -func (se *StorageEndpoint) RemoveScheduleConfig(scheduleName string) error { - return se.Remove(scheduleConfigPath(scheduleName)) +// SaveSchedulerConfig saves the config of the given scheduler. +func (se *StorageEndpoint) SaveSchedulerConfig(schedulerName string, data []byte) error { + return se.Save(schedulerConfigPath(schedulerName), string(data)) +} + +// RemoveSchedulerConfig removes the config of the given scheduler. +func (se *StorageEndpoint) RemoveSchedulerConfig(schedulerName string) error { + return se.Remove(schedulerConfigPath(schedulerName)) } diff --git a/pkg/storage/endpoint/key_path.go b/pkg/storage/endpoint/key_path.go index 4b67441a5ac..cac40db29c5 100644 --- a/pkg/storage/endpoint/key_path.go +++ b/pkg/storage/endpoint/key_path.go @@ -25,17 +25,17 @@ import ( ) const ( - pdRootPath = "/pd" - clusterPath = "raft" - configPath = "config" - serviceMiddlewarePath = "service_middleware" - schedulePath = "schedule" - gcPath = "gc" - rulesPath = "rules" - ruleGroupPath = "rule_group" - regionLabelPath = "region_label" - replicationPath = "replication_mode" - customScheduleConfigPath = "scheduler_config" + pdRootPath = "/pd" + clusterPath = "raft" + configPath = "config" + serviceMiddlewarePath = "service_middleware" + schedulePath = "schedule" + gcPath = "gc" + rulesPath = "rules" + ruleGroupPath = "rule_group" + regionLabelPath = "region_label" + replicationPath = "replication_mode" + customSchedulerConfigPath = "scheduler_config" // GCWorkerServiceSafePointID is the service id of GC worker. GCWorkerServiceSafePointID = "gc_worker" minResolvedTS = "min_resolved_ts" @@ -94,7 +94,7 @@ func ConfigPath(clusterID uint64) string { // SchedulerConfigPathPrefix returns the path prefix to save the scheduler config. func SchedulerConfigPathPrefix(clusterID uint64) string { - return path.Join(PDRootPath(clusterID), customScheduleConfigPath) + return path.Join(PDRootPath(clusterID), customSchedulerConfigPath) } // RulesPathPrefix returns the path prefix to save the placement rules. @@ -112,8 +112,8 @@ func RegionLabelPathPrefix(clusterID uint64) string { return path.Join(PDRootPath(clusterID), regionLabelPath) } -func scheduleConfigPath(scheduleName string) string { - return path.Join(customScheduleConfigPath, scheduleName) +func schedulerConfigPath(schedulerName string) string { + return path.Join(customSchedulerConfigPath, schedulerName) } // StorePath returns the store meta info key path with the given store ID. diff --git a/plugin/scheduler_example/evict_leader.go b/plugin/scheduler_example/evict_leader.go index 91b9e518089..8919d1bdb4b 100644 --- a/plugin/scheduler_example/evict_leader.go +++ b/plugin/scheduler_example/evict_leader.go @@ -133,7 +133,7 @@ func (conf *evictLeaderSchedulerConfig) Persist() error { if err != nil { return err } - return conf.storage.SaveScheduleConfig(name, data) + return conf.storage.SaveSchedulerConfig(name, data) } func (conf *evictLeaderSchedulerConfig) getScheduleName() string { diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 605fd222502..ea8d27b155f 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -3088,7 +3088,7 @@ func TestPersistScheduler(t *testing.T) { re.NoError(err) re.NoError(controller.AddScheduler(evict, "2")) re.Len(controller.GetSchedulerNames(), defaultCount+2) - sches, _, err := storage.LoadAllScheduleConfig() + sches, _, err := storage.LoadAllSchedulerConfigs() re.NoError(err) re.Len(sches, defaultCount+2) @@ -3117,7 +3117,7 @@ func TestPersistScheduler(t *testing.T) { re.Len(newOpt.GetSchedulers(), defaultCount) re.NoError(newOpt.Reload(storage)) // only remains 3 items with independent config. - sches, _, err = storage.LoadAllScheduleConfig() + sches, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) re.Len(sches, 3) @@ -3198,7 +3198,7 @@ func TestRemoveScheduler(t *testing.T) { re.NoError(err) re.NoError(controller.AddScheduler(gls1, "1")) re.Len(controller.GetSchedulerNames(), defaultCount+1) - sches, _, err := storage.LoadAllScheduleConfig() + sches, _, err := storage.LoadAllSchedulerConfigs() re.NoError(err) re.Len(sches, defaultCount+1) @@ -3210,7 +3210,7 @@ func TestRemoveScheduler(t *testing.T) { re.NoError(controller.RemoveScheduler(schedulers.BalanceWitnessName)) re.NoError(controller.RemoveScheduler(schedulers.TransferWitnessLeaderName)) // all removed - sches, _, err = storage.LoadAllScheduleConfig() + sches, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) re.Empty(sches) re.Empty(controller.GetSchedulerNames()) diff --git a/server/server.go b/server/server.go index 2a076923caf..7c19d8ff7c5 100644 --- a/server/server.go +++ b/server/server.go @@ -938,7 +938,7 @@ func (s *Server) GetConfig() *config.Config { if s.storage == nil { return cfg } - sches, configs, err := s.storage.LoadAllScheduleConfig() + sches, configs, err := s.storage.LoadAllSchedulerConfigs() if err != nil { return cfg } diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index 14c99baa6f5..8b8e284f765 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -147,14 +147,14 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { ) re.NoError(err) // Get all default scheduler names. - var namesFromAPIServer, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllScheduleConfig() + var namesFromAPIServer, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllSchedulerConfigs() testutil.Eventually(re, func() bool { return len(namesFromAPIServer) == len(sc.DefaultSchedulers) }) // Check all default schedulers' configs. var namesFromSchedulingServer []string testutil.Eventually(re, func() bool { - namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + namesFromSchedulingServer, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) return len(namesFromSchedulingServer) == len(namesFromAPIServer) }) @@ -165,10 +165,11 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { }) // Check the new scheduler's config. testutil.Eventually(re, func() bool { - namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + namesFromSchedulingServer, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) return slice.Contains(namesFromSchedulingServer, schedulers.EvictLeaderName) }) + assertEvictLeaderStoreIDs(re, storage, []uint64{1}) // Update the scheduler by adding a store. err = suite.pdLeaderServer.GetServer().GetRaftCluster().PutStore( &metapb.Store{ @@ -192,7 +193,7 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { api.MustDeleteScheduler(re, suite.pdLeaderServer.GetAddr(), schedulers.EvictLeaderName) // Check the removed scheduler's config. testutil.Eventually(re, func() bool { - namesFromSchedulingServer, _, err = storage.LoadAllScheduleConfig() + namesFromSchedulingServer, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) return !slice.Contains(namesFromSchedulingServer, schedulers.EvictLeaderName) }) @@ -202,24 +203,15 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { func assertEvictLeaderStoreIDs( re *require.Assertions, storage *endpoint.StorageEndpoint, storeIDs []uint64, ) { - var ( - namesFromSchedulingServer, configs []string - err error - evictLeaderCfg struct { - StoreIDWithRanges map[uint64][]core.KeyRange `json:"store-id-ranges"` - } - ) + var evictLeaderCfg struct { + StoreIDWithRanges map[uint64][]core.KeyRange `json:"store-id-ranges"` + } testutil.Eventually(re, func() bool { - namesFromSchedulingServer, configs, err = storage.LoadAllScheduleConfig() + cfg, err := storage.LoadSchedulerConfig(schedulers.EvictLeaderName) + re.NoError(err) + err = schedulers.DecodeConfig([]byte(cfg), &evictLeaderCfg) re.NoError(err) - for idx, name := range namesFromSchedulingServer { - if name == schedulers.EvictLeaderName { - err = schedulers.DecodeConfig([]byte(configs[idx]), &evictLeaderCfg) - re.NoError(err) - return len(evictLeaderCfg.StoreIDWithRanges) == len(storeIDs) - } - } - return false + return len(evictLeaderCfg.StoreIDWithRanges) == len(storeIDs) }) // Validate the updated scheduler's config. for _, storeID := range storeIDs { diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index f4785a013d9..187ba54dfcb 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -16,6 +16,7 @@ package scheduling import ( "context" + "fmt" "testing" "time" @@ -24,9 +25,11 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/suite" mcs "github.com/tikv/pd/pkg/mcs/utils" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" "github.com/tikv/pd/tests" + "github.com/tikv/pd/tests/server/api" "go.uber.org/goleak" ) @@ -104,6 +107,9 @@ func (suite *serverTestSuite) TestAllocIDAfterLeaderChange() { re.NoError(err) re.Greater(id1, id) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/scheduling/server/fastUpdateMember")) + // Update the pdLeader in test suite. + suite.pdLeader = suite.cluster.GetServer(suite.cluster.WaitLeader()) + suite.backendEndpoints = suite.pdLeader.GetAddr() } func (suite *serverTestSuite) TestPrimaryChange() { @@ -182,3 +188,66 @@ func (suite *serverTestSuite) TestForwardStoreHeartbeat() { store.GetStoreStats().GetBytesRead() == uint64(99) }) } + +func (suite *serverTestSuite) TestSchedulerSync() { + re := suite.Require() + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + schedulersController := tc.GetPrimaryServer().GetCluster().GetCoordinator().GetSchedulersController() + re.Len(schedulersController.GetSchedulerNames(), 5) + re.Nil(schedulersController.GetScheduler(schedulers.EvictLeaderName)) + // Add a new evict-leader-scheduler through the API server. + api.MustAddScheduler(re, suite.backendEndpoints, schedulers.EvictLeaderName, map[string]interface{}{ + "store_id": 1, + }) + // Check if the evict-leader-scheduler is added. + testutil.Eventually(re, func() bool { + return len(schedulersController.GetSchedulerNames()) == 6 && + schedulersController.GetScheduler(schedulers.EvictLeaderName) != nil + }) + handler, ok := schedulersController.GetSchedulerHandlers()[schedulers.EvictLeaderName] + re.True(ok) + h, ok := handler.(interface { + EvictStoreIDs() []uint64 + }) + re.True(ok) + re.ElementsMatch(h.EvictStoreIDs(), []uint64{1}) + // Update the evict-leader-scheduler through the API server. + err = suite.pdLeader.GetServer().GetRaftCluster().PutStore( + &metapb.Store{ + Id: 2, + Address: "mock://2", + State: metapb.StoreState_Up, + NodeState: metapb.NodeState_Serving, + LastHeartbeat: time.Now().UnixNano(), + Version: "7.0.0", + }, + ) + re.NoError(err) + api.MustAddScheduler(re, suite.backendEndpoints, schedulers.EvictLeaderName, map[string]interface{}{ + "store_id": 2, + }) + var evictStoreIDs []uint64 + testutil.Eventually(re, func() bool { + evictStoreIDs = h.EvictStoreIDs() + return len(evictStoreIDs) == 2 + }) + re.ElementsMatch(evictStoreIDs, []uint64{1, 2}) + api.MustDeleteScheduler(re, suite.backendEndpoints, fmt.Sprintf("%s-%d", schedulers.EvictLeaderName, 1)) + testutil.Eventually(re, func() bool { + evictStoreIDs = h.EvictStoreIDs() + return len(evictStoreIDs) == 1 + }) + re.ElementsMatch(evictStoreIDs, []uint64{2}) + // Remove the evict-leader-scheduler through the API server. + api.MustDeleteScheduler(re, suite.backendEndpoints, schedulers.EvictLeaderName) + // Check if the scheduler is removed. + testutil.Eventually(re, func() bool { + return len(schedulersController.GetSchedulerNames()) == 5 && + schedulersController.GetScheduler(schedulers.EvictLeaderName) == nil + }) + + // TODO: test more schedulers. +} From 4eb9aea558c2144d30cda514c95953c70d8cd3ab Mon Sep 17 00:00:00 2001 From: JmPotato Date: Fri, 15 Sep 2023 11:57:08 +0800 Subject: [PATCH 06/10] keyspace: return error if the split keyspace list is empty (#7102) close tikv/pd#7101 Signed-off-by: JmPotato --- pkg/keyspace/tso_keyspace_group.go | 4 ++++ pkg/keyspace/tso_keyspace_group_test.go | 3 +-- pkg/keyspace/util.go | 2 ++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index ac5b035ae3f..a092e5b18a6 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -675,6 +675,10 @@ func buildSplitKeyspaces( newKeyspaceMap[keyspace] = struct{}{} } } + // Check if the new keyspace list is empty. + if len(newSplit) == 0 { + return nil, nil, ErrKeyspaceGroupWithEmptyKeyspace + } // Get the split keyspace list for the old keyspace group. oldSplit := make([]uint32, 0, oldNum-len(newSplit)) for _, keyspace := range old { diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 5f01146eb96..993923d2fd7 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -542,8 +542,7 @@ func TestBuildSplitKeyspaces(t *testing.T) { old: []uint32{1, 2, 3, 4, 5}, startKeyspaceID: 7, endKeyspaceID: 10, - expectedOld: []uint32{1, 2, 3, 4, 5}, - expectedNew: []uint32{}, + err: ErrKeyspaceGroupWithEmptyKeyspace, }, { old: []uint32{1, 2, 3, 4, 5}, diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 6042a0b23be..aa8d0f350ea 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -76,6 +76,8 @@ var ( ErrNodeNotInKeyspaceGroup = errors.New("the tso node is not in this keyspace group") // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. ErrKeyspaceGroupNotEnoughReplicas = errors.New("not enough replicas in the keyspace group") + // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. + ErrKeyspaceGroupWithEmptyKeyspace = errors.New("keyspace group with empty keyspace") // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. ErrModifyDefaultKeyspaceGroup = errors.New("default keyspace group cannot be modified") // ErrNoAvailableNode is used to indicate no available node in the keyspace group. From d7d475679ea435ac324c3599972a752a36425606 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Fri, 15 Sep 2023 15:58:39 +0800 Subject: [PATCH 07/10] mcs: update store stats (#7097) ref tikv/pd#5839 Signed-off-by: Ryan Leung --- pkg/core/basic_cluster.go | 12 +++++++ pkg/mcs/scheduling/server/cluster.go | 48 +++++++++++++++++++++++++--- pkg/mcs/scheduling/server/server.go | 3 +- server/cluster/cluster.go | 8 +---- 4 files changed, 59 insertions(+), 12 deletions(-) diff --git a/pkg/core/basic_cluster.go b/pkg/core/basic_cluster.go index 1c8902ca8cb..2258a816324 100644 --- a/pkg/core/basic_cluster.go +++ b/pkg/core/basic_cluster.go @@ -246,6 +246,18 @@ func (bc *BasicCluster) GetStoresWriteRate() (storeIDs []uint64, bytesRates, key return bc.getWriteRate(bc.RegionsInfo.GetStoreWriteRate) } +// UpdateAllStoreStatus updates the information of all stores. +func (bc *BasicCluster) UpdateAllStoreStatus() { + // Update related stores. + stores := bc.GetStores() + for _, store := range stores { + if store.IsRemoved() { + continue + } + bc.UpdateStoreStatus(store.GetID()) + } +} + // RegionSetInformer provides access to a shared informer of regions. type RegionSetInformer interface { GetTotalRegionCount() int diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 61c85ba6fbd..917831ba9ca 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -2,6 +2,7 @@ package server import ( "context" + "sync" "sync/atomic" "time" @@ -29,7 +30,9 @@ import ( // Cluster is used to manage all information for scheduling purpose. type Cluster struct { - ctx context.Context + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup *core.BasicCluster persistConfig *config.PersistConfig ruleManager *placement.RuleManager @@ -47,14 +50,17 @@ type Cluster struct { const regionLabelGCInterval = time.Hour // NewCluster creates a new cluster. -func NewCluster(ctx context.Context, persistConfig *config.PersistConfig, storage storage.Storage, basicCluster *core.BasicCluster, hbStreams *hbstream.HeartbeatStreams, clusterID uint64, checkMembershipCh chan struct{}) (*Cluster, error) { +func NewCluster(parentCtx context.Context, persistConfig *config.PersistConfig, storage storage.Storage, basicCluster *core.BasicCluster, hbStreams *hbstream.HeartbeatStreams, clusterID uint64, checkMembershipCh chan struct{}) (*Cluster, error) { + ctx, cancel := context.WithCancel(parentCtx) labelerManager, err := labeler.NewRegionLabeler(ctx, storage, regionLabelGCInterval) if err != nil { + cancel() return nil, err } ruleManager := placement.NewRuleManager(storage, basicCluster, persistConfig) c := &Cluster{ ctx: ctx, + cancel: cancel, BasicCluster: basicCluster, ruleManager: ruleManager, labelerManager: labelerManager, @@ -69,6 +75,7 @@ func NewCluster(ctx context.Context, persistConfig *config.PersistConfig, storag c.coordinator = schedule.NewCoordinator(ctx, c, hbStreams) err = c.ruleManager.Initialize(persistConfig.GetMaxReplicas(), persistConfig.GetLocationLabels()) if err != nil { + cancel() return nil, err } return c, nil @@ -179,9 +186,10 @@ func (c *Cluster) SwitchAPIServerLeader(new pdpb.PDClient) bool { return c.apiServerLeader.CompareAndSwap(old, new) } -// UpdateScheduler listens on the schedulers updating notifier and manage the scheduler creation and deletion. -func (c *Cluster) UpdateScheduler() { +// updateScheduler listens on the schedulers updating notifier and manage the scheduler creation and deletion. +func (c *Cluster) updateScheduler() { defer logutil.LogPanic() + defer c.wg.Done() // Make sure the coordinator has initialized all the existing schedulers. c.waitSchedulersInitialized() @@ -348,3 +356,35 @@ func (c *Cluster) HandleStoreHeartbeat(heartbeat *schedulingpb.StoreHeartbeatReq c.hotStat.CheckReadAsync(statistics.NewCollectUnReportedPeerTask(storeID, regions, interval)) return nil } + +// runUpdateStoreStats updates store stats periodically. +func (c *Cluster) runUpdateStoreStats() { + defer logutil.LogPanic() + defer c.wg.Done() + + ticker := time.NewTicker(9 * time.Millisecond) + defer ticker.Stop() + + for { + select { + case <-c.ctx.Done(): + log.Info("update store stats background jobs has been stopped") + return + case <-ticker.C: + c.UpdateAllStoreStatus() + } + } +} + +// StartBackgroundJobs starts background jobs. +func (c *Cluster) StartBackgroundJobs() { + c.wg.Add(2) + go c.updateScheduler() + go c.runUpdateStoreStats() +} + +// StopBackgroundJobs stops background jobs. +func (c *Cluster) StopBackgroundJobs() { + c.cancel() + c.wg.Wait() +} diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 330085e3b82..f4c5c676dd3 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -443,13 +443,14 @@ func (s *Server) startCluster(context.Context) error { return err } s.configWatcher.SetSchedulersController(s.cluster.GetCoordinator().GetSchedulersController()) - go s.cluster.UpdateScheduler() + s.cluster.StartBackgroundJobs() go s.GetCoordinator().RunUntilStop() return nil } func (s *Server) stopCluster() { s.GetCoordinator().Stop() + s.cluster.StopBackgroundJobs() s.ruleWatcher.Close() s.configWatcher.Close() s.metaWatcher.Close() diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 18bf66b8188..29a8709bdac 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -684,13 +684,7 @@ func (c *RaftCluster) runUpdateStoreStats() { case <-ticker.C: // Update related stores. start := time.Now() - stores := c.GetStores() - for _, store := range stores { - if store.IsRemoved() { - continue - } - c.core.UpdateStoreStatus(store.GetID()) - } + c.core.UpdateAllStoreStatus() updateStoreStatsGauge.Set(time.Since(start).Seconds()) } } From 0888ef6fe7824adfee983901bd18c8d001478be2 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Mon, 18 Sep 2023 16:49:41 +0800 Subject: [PATCH 08/10] mcs: forward current http request to mcs (#7078) ref tikv/pd#5839 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/mcs/resourcemanager/server/apis/v1/api.go | 2 +- pkg/mcs/scheduling/server/apis/v1/api.go | 2 +- pkg/utils/apiutil/apiutil.go | 17 +++++ pkg/utils/apiutil/serverapi/middleware.go | 35 +++++++-- pkg/utils/testutil/api_check.go | 73 +++++++++++-------- server/api/hot_status_test.go | 5 +- server/api/region_test.go | 2 +- server/api/rule_test.go | 2 +- server/api/server.go | 34 +++++++-- tests/integrations/mcs/scheduling/api_test.go | 58 +++++++++++++++ tests/integrations/mcs/tso/api_test.go | 32 ++++++-- tests/integrations/mcs/tso/server_test.go | 30 ++++---- tests/pdctl/operator/operator_test.go | 30 ++++++++ tests/pdctl/scheduler/scheduler_test.go | 25 +++++++ 14 files changed, 278 insertions(+), 69 deletions(-) diff --git a/pkg/mcs/resourcemanager/server/apis/v1/api.go b/pkg/mcs/resourcemanager/server/apis/v1/api.go index 411933e55c3..7c5e3e010dc 100644 --- a/pkg/mcs/resourcemanager/server/apis/v1/api.go +++ b/pkg/mcs/resourcemanager/server/apis/v1/api.go @@ -73,7 +73,7 @@ func NewService(srv *rmserver.Service) *Service { manager := srv.GetManager() apiHandlerEngine.Use(func(c *gin.Context) { // manager implements the interface of basicserver.Service. - c.Set("service", manager.GetBasicServer()) + c.Set(multiservicesapi.ServiceContextKey, manager.GetBasicServer()) c.Next() }) apiHandlerEngine.Use(multiservicesapi.ServiceRedirector()) diff --git a/pkg/mcs/scheduling/server/apis/v1/api.go b/pkg/mcs/scheduling/server/apis/v1/api.go index e8c4faa5d55..3d1c3921470 100644 --- a/pkg/mcs/scheduling/server/apis/v1/api.go +++ b/pkg/mcs/scheduling/server/apis/v1/api.go @@ -34,7 +34,7 @@ import ( ) // APIPathPrefix is the prefix of the API path. -const APIPathPrefix = "/scheduling/api/v1/" +const APIPathPrefix = "/scheduling/api/v1" var ( once sync.Once diff --git a/pkg/utils/apiutil/apiutil.go b/pkg/utils/apiutil/apiutil.go index 6c32640218e..0b72b9af10f 100644 --- a/pkg/utils/apiutil/apiutil.go +++ b/pkg/utils/apiutil/apiutil.go @@ -57,6 +57,8 @@ const ( XForwardedPortHeader = "X-Forwarded-Port" // XRealIPHeader is used to mark the real client IP. XRealIPHeader = "X-Real-Ip" + // ForwardToMicroServiceHeader is used to mark the request is forwarded to micro service. + ForwardToMicroServiceHeader = "Forward-To-Micro-Service" // ErrRedirectFailed is the error message for redirect failed. ErrRedirectFailed = "redirect failed" @@ -435,8 +437,17 @@ func (p *customReverseProxies) ServeHTTP(w http.ResponseWriter, r *http.Request) reader = resp.Body } + // We need to copy the response headers before we write the header. + // Otherwise, we cannot set the header after w.WriteHeader() is called. + // And we need to write the header before we copy the response body. + // Otherwise, we cannot set the status code after w.Write() is called. + // In other words, we must perform the following steps strictly in order: + // 1. Set the response headers. + // 2. Write the response header. + // 3. Write the response body. copyHeader(w.Header(), resp.Header) w.WriteHeader(resp.StatusCode) + for { if _, err = io.CopyN(w, reader, chunkSize); err != nil { if err == io.EOF { @@ -455,8 +466,14 @@ func (p *customReverseProxies) ServeHTTP(w http.ResponseWriter, r *http.Request) http.Error(w, ErrRedirectFailed, http.StatusInternalServerError) } +// copyHeader duplicates the HTTP headers from the source `src` to the destination `dst`. +// It skips the "Content-Encoding" and "Content-Length" headers because they should be set by `http.ResponseWriter`. +// These headers may be modified after a redirect when gzip compression is enabled. func copyHeader(dst, src http.Header) { for k, vv := range src { + if k == "Content-Encoding" || k == "Content-Length" { + continue + } values := dst[k] for _, v := range vv { if !slice.Contains(values, v) { diff --git a/pkg/utils/apiutil/serverapi/middleware.go b/pkg/utils/apiutil/serverapi/middleware.go index 7d403ecef13..063ad042dbb 100644 --- a/pkg/utils/apiutil/serverapi/middleware.go +++ b/pkg/utils/apiutil/serverapi/middleware.go @@ -17,9 +17,12 @@ package serverapi import ( "net/http" "net/url" + "strings" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" "github.com/urfave/negroni" @@ -75,6 +78,7 @@ type microserviceRedirectRule struct { matchPath string targetPath string targetServiceName string + matchMethods []string } // NewRedirector redirects request to the leader if needs to be handled in the leader. @@ -90,12 +94,13 @@ func NewRedirector(s *server.Server, opts ...RedirectorOption) negroni.Handler { type RedirectorOption func(*redirector) // MicroserviceRedirectRule new a microservice redirect rule option -func MicroserviceRedirectRule(matchPath, targetPath, targetServiceName string) RedirectorOption { +func MicroserviceRedirectRule(matchPath, targetPath, targetServiceName string, methods []string) RedirectorOption { return func(s *redirector) { s.microserviceRedirectRules = append(s.microserviceRedirectRules, µserviceRedirectRule{ matchPath, targetPath, targetServiceName, + methods, }) } } @@ -108,13 +113,24 @@ func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, stri return false, "" } for _, rule := range h.microserviceRedirectRules { - if rule.matchPath == r.URL.Path { + if strings.HasPrefix(r.URL.Path, rule.matchPath) && slice.Contains(rule.matchMethods, r.Method) { addr, ok := h.s.GetServicePrimaryAddr(r.Context(), rule.targetServiceName) if !ok || addr == "" { - log.Warn("failed to get the service primary addr when try match redirect rules", + log.Warn("failed to get the service primary addr when trying to match redirect rules", zap.String("path", r.URL.Path)) } - r.URL.Path = rule.targetPath + // Extract parameters from the URL path + // e.g. r.URL.Path = /pd/api/v1/operators/1 (before redirect) + // matchPath = /pd/api/v1/operators + // targetPath = /scheduling/api/v1/operators + // r.URL.Path = /scheduling/api/v1/operator/1 (after redirect) + pathParams := strings.TrimPrefix(r.URL.Path, rule.matchPath) + pathParams = strings.Trim(pathParams, "/") // Remove leading and trailing '/' + if len(pathParams) > 0 { + r.URL.Path = rule.targetPath + "/" + pathParams + } else { + r.URL.Path = rule.targetPath + } return true, addr } } @@ -122,10 +138,10 @@ func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, stri } func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http.HandlerFunc) { - matchedFlag, targetAddr := h.matchMicroServiceRedirectRules(r) + redirectToMicroService, targetAddr := h.matchMicroServiceRedirectRules(r) allowFollowerHandle := len(r.Header.Get(apiutil.PDAllowFollowerHandleHeader)) > 0 isLeader := h.s.GetMember().IsLeader() - if !h.s.IsClosed() && (allowFollowerHandle || isLeader) && !matchedFlag { + if !h.s.IsClosed() && (allowFollowerHandle || isLeader) && !redirectToMicroService { next(w, r) return } @@ -150,12 +166,17 @@ func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http } var clientUrls []string - if matchedFlag { + if redirectToMicroService { if len(targetAddr) == 0 { http.Error(w, apiutil.ErrRedirectFailed, http.StatusInternalServerError) return } clientUrls = append(clientUrls, targetAddr) + failpoint.Inject("checkHeader", func() { + // add a header to the response, this is not a failure injection + // it is used for testing, to check whether the request is forwarded to the micro service + w.Header().Set(apiutil.ForwardToMicroServiceHeader, "true") + }) } else { leader := h.s.GetMember().GetLeader() if leader == nil { diff --git a/pkg/utils/testutil/api_check.go b/pkg/utils/testutil/api_check.go index fcc445b7e7a..d11d575967d 100644 --- a/pkg/utils/testutil/api_check.go +++ b/pkg/utils/testutil/api_check.go @@ -23,56 +23,71 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" ) -// Status is used to check whether http response code is equal given code -func Status(re *require.Assertions, code int) func([]byte, int) { - return func(resp []byte, i int) { +// Status is used to check whether http response code is equal given code. +func Status(re *require.Assertions, code int) func([]byte, int, http.Header) { + return func(resp []byte, i int, _ http.Header) { re.Equal(code, i, "resp: "+string(resp)) } } -// StatusOK is used to check whether http response code is equal http.StatusOK -func StatusOK(re *require.Assertions) func([]byte, int) { +// StatusOK is used to check whether http response code is equal http.StatusOK. +func StatusOK(re *require.Assertions) func([]byte, int, http.Header) { return Status(re, http.StatusOK) } -// StatusNotOK is used to check whether http response code is not equal http.StatusOK -func StatusNotOK(re *require.Assertions) func([]byte, int) { - return func(_ []byte, i int) { +// StatusNotOK is used to check whether http response code is not equal http.StatusOK. +func StatusNotOK(re *require.Assertions) func([]byte, int, http.Header) { + return func(_ []byte, i int, _ http.Header) { re.NotEqual(http.StatusOK, i) } } -// ExtractJSON is used to check whether given data can be extracted successfully -func ExtractJSON(re *require.Assertions, data interface{}) func([]byte, int) { - return func(res []byte, _ int) { +// ExtractJSON is used to check whether given data can be extracted successfully. +func ExtractJSON(re *require.Assertions, data interface{}) func([]byte, int, http.Header) { + return func(res []byte, _ int, _ http.Header) { re.NoError(json.Unmarshal(res, data)) } } -// StringContain is used to check whether response context contains given string -func StringContain(re *require.Assertions, sub string) func([]byte, int) { - return func(res []byte, _ int) { +// StringContain is used to check whether response context contains given string. +func StringContain(re *require.Assertions, sub string) func([]byte, int, http.Header) { + return func(res []byte, _ int, _ http.Header) { re.Contains(string(res), sub) } } -// StringEqual is used to check whether response context equal given string -func StringEqual(re *require.Assertions, str string) func([]byte, int) { - return func(res []byte, _ int) { +// StringEqual is used to check whether response context equal given string. +func StringEqual(re *require.Assertions, str string) func([]byte, int, http.Header) { + return func(res []byte, _ int, _ http.Header) { re.Contains(string(res), str) } } -// ReadGetJSON is used to do get request and check whether given data can be extracted successfully -func ReadGetJSON(re *require.Assertions, client *http.Client, url string, data interface{}) error { +// WithHeader is used to check whether response header contains given key and value. +func WithHeader(re *require.Assertions, key, value string) func([]byte, int, http.Header) { + return func(_ []byte, _ int, header http.Header) { + re.Equal(value, header.Get(key)) + } +} + +// WithoutHeader is used to check whether response header does not contain given key. +func WithoutHeader(re *require.Assertions, key string) func([]byte, int, http.Header) { + return func(_ []byte, _ int, header http.Header) { + re.Empty(header.Get(key)) + } +} + +// ReadGetJSON is used to do get request and check whether given data can be extracted successfully. +func ReadGetJSON(re *require.Assertions, client *http.Client, url string, data interface{}, checkOpts ...func([]byte, int, http.Header)) error { resp, err := apiutil.GetJSON(client, url, nil) if err != nil { return err } - return checkResp(resp, StatusOK(re), ExtractJSON(re, data)) + checkOpts = append(checkOpts, StatusOK(re), ExtractJSON(re, data)) + return checkResp(resp, checkOpts...) } -// ReadGetJSONWithBody is used to do get request with input and check whether given data can be extracted successfully +// ReadGetJSONWithBody is used to do get request with input and check whether given data can be extracted successfully. func ReadGetJSONWithBody(re *require.Assertions, client *http.Client, url string, input []byte, data interface{}) error { resp, err := apiutil.GetJSON(client, url, input) if err != nil { @@ -81,8 +96,8 @@ func ReadGetJSONWithBody(re *require.Assertions, client *http.Client, url string return checkResp(resp, StatusOK(re), ExtractJSON(re, data)) } -// CheckPostJSON is used to do post request and do check options -func CheckPostJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int)) error { +// CheckPostJSON is used to do post request and do check options. +func CheckPostJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int, http.Header)) error { resp, err := apiutil.PostJSON(client, url, data) if err != nil { return err @@ -90,8 +105,8 @@ func CheckPostJSON(client *http.Client, url string, data []byte, checkOpts ...fu return checkResp(resp, checkOpts...) } -// CheckGetJSON is used to do get request and do check options -func CheckGetJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int)) error { +// CheckGetJSON is used to do get request and do check options. +func CheckGetJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int, http.Header)) error { resp, err := apiutil.GetJSON(client, url, data) if err != nil { return err @@ -99,8 +114,8 @@ func CheckGetJSON(client *http.Client, url string, data []byte, checkOpts ...fun return checkResp(resp, checkOpts...) } -// CheckPatchJSON is used to do patch request and do check options -func CheckPatchJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int)) error { +// CheckPatchJSON is used to do patch request and do check options. +func CheckPatchJSON(client *http.Client, url string, data []byte, checkOpts ...func([]byte, int, http.Header)) error { resp, err := apiutil.PatchJSON(client, url, data) if err != nil { return err @@ -108,14 +123,14 @@ func CheckPatchJSON(client *http.Client, url string, data []byte, checkOpts ...f return checkResp(resp, checkOpts...) } -func checkResp(resp *http.Response, checkOpts ...func([]byte, int)) error { +func checkResp(resp *http.Response, checkOpts ...func([]byte, int, http.Header)) error { res, err := io.ReadAll(resp.Body) resp.Body.Close() if err != nil { return err } for _, opt := range checkOpts { - opt(res, resp.StatusCode) + opt(res, resp.StatusCode, resp.Header) } return nil } diff --git a/server/api/hot_status_test.go b/server/api/hot_status_test.go index a1d1bbc2617..d3d495f86fa 100644 --- a/server/api/hot_status_test.go +++ b/server/api/hot_status_test.go @@ -17,6 +17,7 @@ package api import ( "encoding/json" "fmt" + "net/http" "testing" "time" @@ -92,7 +93,7 @@ func (suite *hotStatusTestSuite) TestGetHistoryHotRegionsTimeRange() { StartTime: now.UnixNano() / int64(time.Millisecond), EndTime: now.Add(10*time.Second).UnixNano() / int64(time.Millisecond), } - check := func(res []byte, statusCode int) { + check := func(res []byte, statusCode int, _ http.Header) { suite.Equal(200, statusCode) historyHotRegions := &storage.HistoryHotRegions{} json.Unmarshal(res, historyHotRegions) @@ -177,7 +178,7 @@ func (suite *hotStatusTestSuite) TestGetHistoryHotRegionsIDAndTypes() { IsLearners: []bool{false}, EndTime: now.Add(10*time.Minute).UnixNano() / int64(time.Millisecond), } - check := func(res []byte, statusCode int) { + check := func(res []byte, statusCode int, _ http.Header) { suite.Equal(200, statusCode) historyHotRegions := &storage.HistoryHotRegions{} json.Unmarshal(res, historyHotRegions) diff --git a/server/api/region_test.go b/server/api/region_test.go index 63da19ab082..acd305884d4 100644 --- a/server/api/region_test.go +++ b/server/api/region_test.go @@ -408,7 +408,7 @@ func (suite *regionTestSuite) TestSplitRegions() { hex.EncodeToString([]byte("bbb")), hex.EncodeToString([]byte("ccc")), hex.EncodeToString([]byte("ddd"))) - checkOpt := func(res []byte, code int) { + checkOpt := func(res []byte, code int, _ http.Header) { s := &struct { ProcessedPercentage int `json:"processed-percentage"` NewRegionsID []uint64 `json:"regions-id"` diff --git a/server/api/rule_test.go b/server/api/rule_test.go index d2000eb9562..4cea1523401 100644 --- a/server/api/rule_test.go +++ b/server/api/rule_test.go @@ -829,7 +829,7 @@ func (suite *ruleTestSuite) TestBundleBadRequest() { } for _, testCase := range testCases { err := tu.CheckPostJSON(testDialClient, suite.urlPrefix+testCase.uri, []byte(testCase.data), - func(_ []byte, code int) { + func(_ []byte, code int, _ http.Header) { suite.Equal(testCase.ok, code == http.StatusOK) }) suite.NoError(err) diff --git a/server/api/server.go b/server/api/server.go index 1d881022c04..0094d8eb5dd 100644 --- a/server/api/server.go +++ b/server/api/server.go @@ -19,6 +19,7 @@ import ( "net/http" "github.com/gorilla/mux" + scheapi "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" tsoapi "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" mcs "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/apiutil" @@ -35,14 +36,37 @@ func NewHandler(_ context.Context, svr *server.Server) (http.Handler, apiutil.AP Name: "core", IsCore: true, } - router := mux.NewRouter() + prefix := apiPrefix + "/api/v1" r := createRouter(apiPrefix, svr) + router := mux.NewRouter() router.PathPrefix(apiPrefix).Handler(negroni.New( serverapi.NewRuntimeServiceValidator(svr, group), - serverapi.NewRedirector(svr, serverapi.MicroserviceRedirectRule( - apiPrefix+"/api/v1"+"/admin/reset-ts", - tsoapi.APIPathPrefix+"/admin/reset-ts", - mcs.TSOServiceName)), + serverapi.NewRedirector(svr, + serverapi.MicroserviceRedirectRule( + prefix+"/admin/reset-ts", + tsoapi.APIPathPrefix+"/admin/reset-ts", + mcs.TSOServiceName, + []string{http.MethodPost}), + serverapi.MicroserviceRedirectRule( + prefix+"/operators", + scheapi.APIPathPrefix+"/operators", + mcs.SchedulingServiceName, + []string{http.MethodPost, http.MethodGet, http.MethodDelete}), + // because the writing of all the meta information of the scheduling service is in the API server, + // we only forward read-only requests about checkers and schedulers to the scheduling service. + serverapi.MicroserviceRedirectRule( + prefix+"/checker", // Note: this is a typo in the original code + scheapi.APIPathPrefix+"/checkers", + mcs.SchedulingServiceName, + []string{http.MethodGet}), + serverapi.MicroserviceRedirectRule( + prefix+"/schedulers", + scheapi.APIPathPrefix+"/schedulers", + mcs.SchedulingServiceName, + []string{http.MethodGet}), + // TODO: we need to consider the case that v1 api not support restful api. + // we might change the previous path parameters to query parameters. + ), negroni.Wrap(r)), ) diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index 48bdf1ab95c..04671d84798 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -2,13 +2,16 @@ package scheduling_test import ( "context" + "encoding/json" "fmt" "net/http" "testing" "time" + "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" _ "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" @@ -106,3 +109,58 @@ func (suite *apiTestSuite) TestGetCheckerByName() { suite.False(resp["paused"].(bool)) } } + +func (suite *apiTestSuite) TestAPIForward() { + re := suite.Require() + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 2, suite.backendEndpoints) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + + failpoint.Enable("github.com/tikv/pd/pkg/utils/apiutil/serverapi/checkHeader", "return(true)") + defer func() { + failpoint.Disable("github.com/tikv/pd/pkg/utils/apiutil/serverapi/checkHeader") + }() + + urlPrefix := fmt.Sprintf("%s/pd/api/v1", suite.backendEndpoints) + var slice []string + var resp map[string]interface{} + + // Test opeartor + err = testutil.ReadGetJSON(re, testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators"), &slice, + testutil.WithHeader(re, apiutil.ForwardToMicroServiceHeader, "true")) + re.NoError(err) + re.Len(slice, 0) + + err = testutil.ReadGetJSON(re, testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators/2"), &resp, + testutil.WithHeader(re, apiutil.ForwardToMicroServiceHeader, "true")) + re.NoError(err) + re.Nil(resp) + + // Test checker: only read-only requests are forwarded + err = testutil.ReadGetJSON(re, testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "checker/merge"), &resp, + testutil.WithHeader(re, apiutil.ForwardToMicroServiceHeader, "true")) + re.NoError(err) + suite.False(resp["paused"].(bool)) + + input := make(map[string]interface{}) + input["delay"] = 10 + pauseArgs, err := json.Marshal(input) + suite.NoError(err) + err = testutil.CheckPostJSON(testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "checker/merge"), pauseArgs, + testutil.StatusOK(re), testutil.WithoutHeader(re, apiutil.PDRedirectorHeader)) + suite.NoError(err) + + // Test scheduler: only read-only requests are forwarded + err = testutil.ReadGetJSON(re, testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers"), &slice, + testutil.WithHeader(re, apiutil.ForwardToMicroServiceHeader, "true")) + re.NoError(err) + re.Contains(slice, "balance-leader-scheduler") + + input["delay"] = 30 + pauseArgs, err = json.Marshal(input) + suite.NoError(err) + err = testutil.CheckPostJSON(testDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers/all"), pauseArgs, + testutil.StatusOK(re), testutil.WithoutHeader(re, apiutil.ForwardToMicroServiceHeader)) + suite.NoError(err) +} diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index fde6bcb8da0..7e870fbc198 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -30,6 +30,7 @@ import ( apis "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/storage/endpoint" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" ) @@ -47,10 +48,11 @@ var dialClient = &http.Client{ type tsoAPITestSuite struct { suite.Suite - ctx context.Context - cancel context.CancelFunc - pdCluster *tests.TestCluster - tsoCluster *tests.TestTSOCluster + ctx context.Context + cancel context.CancelFunc + pdCluster *tests.TestCluster + tsoCluster *tests.TestTSOCluster + backendEndpoints string } func TestTSOAPI(t *testing.T) { @@ -69,7 +71,8 @@ func (suite *tsoAPITestSuite) SetupTest() { leaderName := suite.pdCluster.WaitLeader() pdLeaderServer := suite.pdCluster.GetServer(leaderName) re.NoError(pdLeaderServer.BootstrapCluster()) - suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 1, pdLeaderServer.GetAddr()) + suite.backendEndpoints = pdLeaderServer.GetAddr() + suite.tsoCluster, err = tests.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) re.NoError(err) } @@ -95,6 +98,25 @@ func (suite *tsoAPITestSuite) TestGetKeyspaceGroupMembers() { re.Equal(primaryMember.GetLeaderID(), defaultGroupMember.PrimaryID) } +func (suite *tsoAPITestSuite) TestForwardResetTS() { + re := suite.Require() + primary := suite.tsoCluster.WaitForDefaultPrimaryServing(re) + re.NotNil(primary) + url := suite.backendEndpoints + "/pd/api/v1/admin/reset-ts" + + // Test reset ts + input := []byte(`{"tso":"121312", "force-use-larger":true}`) + err := testutil.CheckPostJSON(dialClient, url, input, + testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully")) + suite.NoError(err) + + // Test reset ts with invalid tso + input = []byte(`{}`) + err = testutil.CheckPostJSON(dialClient, url, input, + testutil.StatusNotOK(re), testutil.StringContain(re, "invalid tso value")) + re.NoError(err) +} + func mustGetKeyspaceGroupMembers(re *require.Assertions, server *tso.Server) map[uint32]*apis.KeyspaceGroupMember { httpReq, err := http.NewRequest(http.MethodGet, server.GetAddr()+tsoKeyspaceGroupsPrefix+"/members", nil) re.NoError(err) diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index d87f1542179..58006b87eeb 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -106,23 +106,19 @@ func (suite *tsoServerTestSuite) TestTSOServerStartAndStopNormally() { cc, err := grpc.DialContext(suite.ctx, s.GetAddr(), grpc.WithInsecure()) re.NoError(err) cc.Close() - url := s.GetAddr() + tsoapi.APIPathPrefix - { - resetJSON := `{"tso":"121312", "force-use-larger":true}` - re.NoError(err) - resp, err := http.Post(url+"/admin/reset-ts", "application/json", strings.NewReader(resetJSON)) - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusOK, resp.StatusCode) - } - { - resetJSON := `{}` - re.NoError(err) - resp, err := http.Post(url+"/admin/reset-ts", "application/json", strings.NewReader(resetJSON)) - re.NoError(err) - defer resp.Body.Close() - re.Equal(http.StatusBadRequest, resp.StatusCode) - } + + url := s.GetAddr() + tsoapi.APIPathPrefix + "/admin/reset-ts" + // Test reset ts + input := []byte(`{"tso":"121312", "force-use-larger":true}`) + err = testutil.CheckPostJSON(dialClient, url, input, + testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully")) + suite.NoError(err) + + // Test reset ts with invalid tso + input = []byte(`{}`) + err = testutil.CheckPostJSON(dialClient, suite.backendEndpoints+"/pd/api/v1/admin/reset-ts", input, + testutil.StatusNotOK(re), testutil.StringContain(re, "invalid tso value")) + re.NoError(err) } func (suite *tsoServerTestSuite) TestParticipantStartWithAdvertiseListenAddr() { diff --git a/tests/pdctl/operator/operator_test.go b/tests/pdctl/operator/operator_test.go index ab5687cdc04..148cbc9e081 100644 --- a/tests/pdctl/operator/operator_test.go +++ b/tests/pdctl/operator/operator_test.go @@ -17,6 +17,7 @@ package operator_test import ( "context" "encoding/hex" + "encoding/json" "strconv" "strings" "testing" @@ -251,3 +252,32 @@ func TestOperator(t *testing.T) { return strings.Contains(string(output1), "Success!") || strings.Contains(string(output2), "Success!") }) } + +func TestForwardOperatorRequest(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + cluster, err := tests.NewTestAPICluster(ctx, 1) + re.NoError(err) + re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) + server := cluster.GetServer(cluster.GetLeader()) + re.NoError(server.BootstrapCluster()) + backendEndpoints := server.GetAddr() + tc, err := tests.NewTestSchedulingCluster(ctx, 2, backendEndpoints) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + + cmd := pdctlCmd.GetRootCmd() + args := []string{"-u", backendEndpoints, "operator", "show"} + var slice []string + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.NoError(json.Unmarshal(output, &slice)) + re.Len(slice, 0) + args = []string{"-u", backendEndpoints, "operator", "check", "2"} + output, err = pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.Contains(string(output), "null") +} diff --git a/tests/pdctl/scheduler/scheduler_test.go b/tests/pdctl/scheduler/scheduler_test.go index a0447642cb6..31e6270aa3b 100644 --- a/tests/pdctl/scheduler/scheduler_test.go +++ b/tests/pdctl/scheduler/scheduler_test.go @@ -530,3 +530,28 @@ func mightExec(re *require.Assertions, cmd *cobra.Command, args []string, v inte } json.Unmarshal(output, v) } + +func TestForwardSchedulerRequest(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + cluster, err := tests.NewTestAPICluster(ctx, 1) + re.NoError(err) + re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) + server := cluster.GetServer(cluster.GetLeader()) + re.NoError(server.BootstrapCluster()) + backendEndpoints := server.GetAddr() + tc, err := tests.NewTestSchedulingCluster(ctx, 2, backendEndpoints) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + + cmd := pdctlCmd.GetRootCmd() + args := []string{"-u", backendEndpoints, "scheduler", "show"} + var slice []string + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + re.NoError(json.Unmarshal(output, &slice)) + re.Contains(slice, "balance-leader-scheduler") +} From 54eb4e495486cd4e6d1e09a92f09204a0d069184 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Tue, 19 Sep 2023 05:25:40 +0800 Subject: [PATCH 09/10] resource_control: supports dynamically change the controller config (#7042) close tikv/pd#7043 resource_control: supports dynamically change the controller config - supports dynamically changing the controller config - export the `maxWaitDuration` for the local bucket limiter. Signed-off-by: nolouch Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/go.mod | 1 + client/go.sum | 1 + client/resource_group/controller/config.go | 44 +++---- .../resource_group/controller/controller.go | 123 +++++++++++++----- client/resource_group/controller/util.go | 68 ++++++++++ client/resource_group/controller/util_test.go | 51 ++++++++ client/resource_manager_client.go | 10 +- pkg/mcs/resourcemanager/server/apis/v1/api.go | 45 +++++++ pkg/mcs/resourcemanager/server/config.go | 6 + pkg/mcs/resourcemanager/server/config_test.go | 2 + pkg/mcs/resourcemanager/server/manager.go | 72 +++++++++- pkg/storage/endpoint/resource_group.go | 6 + server/server.go | 5 +- .../resourcemanager/resource_manager_test.go | 104 +++++++++++++++ 14 files changed, 473 insertions(+), 65 deletions(-) create mode 100644 client/resource_group/controller/util.go create mode 100644 client/resource_group/controller/util_test.go diff --git a/client/go.mod b/client/go.mod index 9eb066d0fcc..099bcf86296 100644 --- a/client/go.mod +++ b/client/go.mod @@ -3,6 +3,7 @@ module github.com/tikv/pd/client go 1.21 require ( + github.com/BurntSushi/toml v0.3.1 github.com/cloudfoundry/gosigar v1.3.6 github.com/gogo/protobuf v1.3.2 github.com/opentracing/opentracing-go v1.2.0 diff --git a/client/go.sum b/client/go.sum index 33ba3254d53..9261ab4a999 100644 --- a/client/go.sum +++ b/client/go.sum @@ -1,4 +1,5 @@ cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= diff --git a/client/resource_group/controller/config.go b/client/resource_group/controller/config.go index 2095bc60601..16a2525cd0d 100644 --- a/client/resource_group/controller/config.go +++ b/client/resource_group/controller/config.go @@ -51,7 +51,7 @@ const ( // According to the resource control Grafana panel and Prometheus sampling period, the period should be the factor of 15. defaultTargetPeriod = 5 * time.Second // defaultMaxWaitDuration is the max duration to wait for the token before throwing error. - defaultMaxWaitDuration = time.Second + defaultMaxWaitDuration = 30 * time.Second ) const ( @@ -73,14 +73,17 @@ const ( // Because the resource manager has not been deployed in microservice mode, // do not enable this function. - defaultDegradedModeWaitDuration = "0s" + defaultDegradedModeWaitDuration = 0 defaultAvgBatchProportion = 0.7 ) // Config is the configuration of the resource manager controller which includes some option for client needed. type Config struct { // EnableDegradedMode is to control whether resource control client enable degraded mode when server is disconnect. - DegradedModeWaitDuration string `toml:"degraded-mode-wait-duration" json:"degraded-mode-wait-duration"` + DegradedModeWaitDuration Duration `toml:"degraded-mode-wait-duration" json:"degraded-mode-wait-duration"` + + // LTBMaxWaitDuration is the max wait time duration for local token bucket. + LTBMaxWaitDuration Duration `toml:"ltb-max-wait-duration" json:"ltb-max-wait-duration"` // RequestUnit is the configuration determines the coefficients of the RRU and WRU cost. // This configuration should be modified carefully. @@ -90,7 +93,8 @@ type Config struct { // DefaultConfig returns the default resource manager controller configuration. func DefaultConfig() *Config { return &Config{ - DegradedModeWaitDuration: defaultDegradedModeWaitDuration, + DegradedModeWaitDuration: NewDuration(defaultDegradedModeWaitDuration), + LTBMaxWaitDuration: NewDuration(defaultMaxWaitDuration), RequestUnit: DefaultRequestUnitConfig(), } } @@ -143,8 +147,10 @@ type RUConfig struct { WriteBytesCost RequestUnit CPUMsCost RequestUnit // The CPU statistics need to distinguish between different environments. - isSingleGroupByKeyspace bool - maxWaitDuration time.Duration + isSingleGroupByKeyspace bool + + // some config for client + LTBMaxWaitDuration time.Duration DegradedModeWaitDuration time.Duration } @@ -157,21 +163,15 @@ func DefaultRUConfig() *RUConfig { // GenerateRUConfig generates the configuration by the given request unit configuration. func GenerateRUConfig(config *Config) *RUConfig { - cfg := &RUConfig{ - ReadBaseCost: RequestUnit(config.RequestUnit.ReadBaseCost), - ReadPerBatchBaseCost: RequestUnit(config.RequestUnit.ReadPerBatchBaseCost), - ReadBytesCost: RequestUnit(config.RequestUnit.ReadCostPerByte), - WriteBaseCost: RequestUnit(config.RequestUnit.WriteBaseCost), - WritePerBatchBaseCost: RequestUnit(config.RequestUnit.WritePerBatchBaseCost), - WriteBytesCost: RequestUnit(config.RequestUnit.WriteCostPerByte), - CPUMsCost: RequestUnit(config.RequestUnit.CPUMsCost), - maxWaitDuration: defaultMaxWaitDuration, - } - duration, err := time.ParseDuration(config.DegradedModeWaitDuration) - if err != nil { - cfg.DegradedModeWaitDuration, _ = time.ParseDuration(defaultDegradedModeWaitDuration) - } else { - cfg.DegradedModeWaitDuration = duration + return &RUConfig{ + ReadBaseCost: RequestUnit(config.RequestUnit.ReadBaseCost), + ReadPerBatchBaseCost: RequestUnit(config.RequestUnit.ReadPerBatchBaseCost), + ReadBytesCost: RequestUnit(config.RequestUnit.ReadCostPerByte), + WriteBaseCost: RequestUnit(config.RequestUnit.WriteBaseCost), + WritePerBatchBaseCost: RequestUnit(config.RequestUnit.WritePerBatchBaseCost), + WriteBytesCost: RequestUnit(config.RequestUnit.WriteCostPerByte), + CPUMsCost: RequestUnit(config.RequestUnit.CPUMsCost), + LTBMaxWaitDuration: config.LTBMaxWaitDuration.Duration, + DegradedModeWaitDuration: config.DegradedModeWaitDuration.Duration, } - return cfg } diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index c79bfec1e56..528369df229 100755 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -71,9 +71,11 @@ type ResourceGroupProvider interface { ModifyResourceGroup(ctx context.Context, metaGroup *rmpb.ResourceGroup) (string, error) DeleteResourceGroup(ctx context.Context, resourceGroupName string) (string, error) AcquireTokenBuckets(ctx context.Context, request *rmpb.TokenBucketsRequest) ([]*rmpb.TokenBucketResponse, error) - LoadGlobalConfig(ctx context.Context, names []string, configPath string) ([]pd.GlobalConfigItem, int64, error) LoadResourceGroups(ctx context.Context) ([]*rmpb.ResourceGroup, int64, error) + + // meta storage client Watch(ctx context.Context, key []byte, opts ...pd.OpOption) (chan []*meta_storagepb.Event, error) + Get(ctx context.Context, key []byte, opts ...pd.OpOption) (*meta_storagepb.GetResponse, error) } // ResourceControlCreateOption create a ResourceGroupsController with the optional settings. @@ -89,7 +91,7 @@ func EnableSingleGroupByKeyspace() ResourceControlCreateOption { // WithMaxWaitDuration is the option to set the max wait duration for acquiring token buckets. func WithMaxWaitDuration(d time.Duration) ResourceControlCreateOption { return func(controller *ResourceGroupsController) { - controller.ruConfig.maxWaitDuration = d + controller.ruConfig.LTBMaxWaitDuration = d } } @@ -122,6 +124,11 @@ type ResourceGroupsController struct { // Currently, we don't do multiple `AcquireTokenBuckets`` at the same time, so there are no concurrency problems with `currentRequests`. currentRequests []*rmpb.TokenBucketRequest } + + opts []ResourceControlCreateOption + + // a cache for ru config and make concurrency safe. + safeRuConfig atomic.Pointer[RUConfig] } // NewResourceGroupController returns a new ResourceGroupsController which impls ResourceGroupKVInterceptor @@ -139,7 +146,7 @@ func NewResourceGroupController( if requestUnitConfig != nil { config.RequestUnit = *requestUnitConfig } - log.Info("load resource controller config", zap.Reflect("config", config)) + ruConfig := GenerateRUConfig(config) controller := &ResourceGroupsController{ clientUniqueID: clientUniqueID, @@ -148,34 +155,37 @@ func NewResourceGroupController( lowTokenNotifyChan: make(chan struct{}, 1), tokenResponseChan: make(chan []*rmpb.TokenBucketResponse, 1), tokenBucketUpdateChan: make(chan *groupCostController, maxNotificationChanLen), + opts: opts, } for _, opt := range opts { opt(controller) } + log.Info("load resource controller config", zap.Reflect("config", config), zap.Reflect("ru-config", controller.ruConfig)) controller.calculators = []ResourceCalculator{newKVCalculator(controller.ruConfig), newSQLCalculator(controller.ruConfig)} + controller.safeRuConfig.Store(controller.ruConfig) return controller, nil } func loadServerConfig(ctx context.Context, provider ResourceGroupProvider) (*Config, error) { - items, _, err := provider.LoadGlobalConfig(ctx, nil, controllerConfigPath) + resp, err := provider.Get(ctx, []byte(controllerConfigPath)) if err != nil { return nil, err } - if len(items) == 0 { + if len(resp.Kvs) == 0 { log.Warn("[resource group controller] server does not save config, load config failed") return DefaultConfig(), nil } config := &Config{} - err = json.Unmarshal(items[0].PayLoad, config) + err = json.Unmarshal(resp.Kvs[0].GetValue(), config) if err != nil { return nil, err } return config, nil } -// GetConfig returns the config of controller. It's only used for test. +// GetConfig returns the config of controller. func (c *ResourceGroupsController) GetConfig() *RUConfig { - return c.ruConfig + return c.safeRuConfig.Load() } // Source List @@ -213,22 +223,63 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { stateUpdateTicker = time.NewTicker(time.Millisecond * 100) }) - _, revision, err := c.provider.LoadResourceGroups(ctx) + _, metaRevision, err := c.provider.LoadResourceGroups(ctx) + if err != nil { + log.Warn("load resource group revision failed", zap.Error(err)) + } + resp, err := c.provider.Get(ctx, []byte(controllerConfigPath)) if err != nil { log.Warn("load resource group revision failed", zap.Error(err)) } - var watchChannel chan []*meta_storagepb.Event + cfgRevision := resp.GetHeader().GetRevision() + var watchMetaChannel, watchConfigChannel chan []*meta_storagepb.Event if !c.ruConfig.isSingleGroupByKeyspace { - watchChannel, err = c.provider.Watch(ctx, pd.GroupSettingsPathPrefixBytes, pd.WithRev(revision), pd.WithPrefix()) + watchMetaChannel, err = c.provider.Watch(ctx, pd.GroupSettingsPathPrefixBytes, pd.WithRev(metaRevision), pd.WithPrefix()) + if err != nil { + log.Warn("watch resource group meta failed", zap.Error(err)) + } } - watchRetryTimer := time.NewTimer(watchRetryInterval) - if err == nil || c.ruConfig.isSingleGroupByKeyspace { - watchRetryTimer.Stop() + + watchConfigChannel, err = c.provider.Watch(ctx, pd.ControllerConfigPathPrefixBytes, pd.WithRev(cfgRevision), pd.WithPrefix()) + if err != nil { + log.Warn("watch resource group config failed", zap.Error(err)) } + watchRetryTimer := time.NewTimer(watchRetryInterval) defer watchRetryTimer.Stop() for { select { + /* tickers */ + case <-cleanupTicker.C: + c.cleanUpResourceGroup() + case <-stateUpdateTicker.C: + c.executeOnAllGroups((*groupCostController).updateRunState) + c.executeOnAllGroups((*groupCostController).updateAvgRequestResourcePerSec) + if len(c.run.currentRequests) == 0 { + c.collectTokenBucketRequests(c.loopCtx, FromPeriodReport, periodicReport /* select resource groups which should be reported periodically */) + } + case <-watchRetryTimer.C: + if !c.ruConfig.isSingleGroupByKeyspace && watchMetaChannel == nil { + watchMetaChannel, err = c.provider.Watch(ctx, pd.GroupSettingsPathPrefixBytes, pd.WithRev(metaRevision), pd.WithPrefix()) + if err != nil { + log.Warn("watch resource group meta failed", zap.Error(err)) + watchRetryTimer.Reset(watchRetryInterval) + failpoint.Inject("watchStreamError", func() { + watchRetryTimer.Reset(20 * time.Millisecond) + }) + } + } + if watchConfigChannel == nil { + watchConfigChannel, err = c.provider.Watch(ctx, pd.ControllerConfigPathPrefixBytes, pd.WithRev(cfgRevision), pd.WithPrefix()) + if err != nil { + log.Warn("watch resource group config failed", zap.Error(err)) + watchRetryTimer.Reset(watchRetryInterval) + } + } + + case <-emergencyTokenAcquisitionTicker.C: + c.executeOnAllGroups((*groupCostController).resetEmergencyTokenAcquisition) + /* channels */ case <-c.loopCtx.Done(): resourceGroupStatusGauge.Reset() return @@ -242,14 +293,6 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { c.handleTokenBucketResponse(resp) } c.run.currentRequests = nil - case <-cleanupTicker.C: - c.cleanUpResourceGroup() - case <-stateUpdateTicker.C: - c.executeOnAllGroups((*groupCostController).updateRunState) - c.executeOnAllGroups((*groupCostController).updateAvgRequestResourcePerSec) - if len(c.run.currentRequests) == 0 { - c.collectTokenBucketRequests(c.loopCtx, FromPeriodReport, periodicReport /* select resource groups which should be reported periodically */) - } case <-c.lowTokenNotifyChan: c.executeOnAllGroups((*groupCostController).updateRunState) c.executeOnAllGroups((*groupCostController).updateAvgRequestResourcePerSec) @@ -259,16 +302,14 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { if c.run.inDegradedMode { c.executeOnAllGroups((*groupCostController).applyDegradedMode) } - case <-emergencyTokenAcquisitionTicker.C: - c.executeOnAllGroups((*groupCostController).resetEmergencyTokenAcquisition) - case resp, ok := <-watchChannel: + case resp, ok := <-watchMetaChannel: failpoint.Inject("disableWatch", func() { if c.ruConfig.isSingleGroupByKeyspace { panic("disableWatch") } }) if !ok { - watchChannel = nil + watchMetaChannel = nil watchRetryTimer.Reset(watchRetryInterval) failpoint.Inject("watchStreamError", func() { watchRetryTimer.Reset(20 * time.Millisecond) @@ -276,7 +317,7 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { continue } for _, item := range resp { - revision = item.Kv.ModRevision + metaRevision = item.Kv.ModRevision group := &rmpb.ResourceGroup{} if err := proto.Unmarshal(item.Kv.Value, group); err != nil { continue @@ -293,14 +334,32 @@ func (c *ResourceGroupsController) Start(ctx context.Context) { } } } - case <-watchRetryTimer.C: - watchChannel, err = c.provider.Watch(ctx, pd.GroupSettingsPathPrefixBytes, pd.WithRev(revision), pd.WithPrefix()) - if err != nil { + case resp, ok := <-watchConfigChannel: + if !ok { + watchConfigChannel = nil watchRetryTimer.Reset(watchRetryInterval) failpoint.Inject("watchStreamError", func() { watchRetryTimer.Reset(20 * time.Millisecond) }) + continue + } + for _, item := range resp { + cfgRevision = item.Kv.ModRevision + config := &Config{} + if err := json.Unmarshal(item.Kv.Value, config); err != nil { + continue + } + c.ruConfig = GenerateRUConfig(config) + + // Stay compatible with serverless + for _, opt := range c.opts { + opt(c) + } + copyCfg := *c.ruConfig + c.safeRuConfig.Store(©Cfg) + log.Info("load resource controller config after config changed", zap.Reflect("config", config), zap.Reflect("ruConfig", c.ruConfig)) } + case gc := <-c.tokenBucketUpdateChan: now := gc.run.now go gc.handleTokenBucketUpdateEvent(c.loopCtx, now) @@ -1127,7 +1186,7 @@ func (gc *groupCostController) onRequestWait( res := make([]*Reservation, 0, len(requestResourceLimitTypeList)) for typ, counter := range gc.run.resourceTokens { if v := getRawResourceValueFromConsumption(delta, typ); v > 0 { - res = append(res, counter.limiter.Reserve(ctx, gc.mainCfg.maxWaitDuration, now, v)) + res = append(res, counter.limiter.Reserve(ctx, gc.mainCfg.LTBMaxWaitDuration, now, v)) } } if d, err = WaitReservations(ctx, now, res); err == nil { @@ -1137,7 +1196,7 @@ func (gc *groupCostController) onRequestWait( res := make([]*Reservation, 0, len(requestUnitLimitTypeList)) for typ, counter := range gc.run.requestUnitTokens { if v := getRUValueFromConsumption(delta, typ); v > 0 { - res = append(res, counter.limiter.Reserve(ctx, gc.mainCfg.maxWaitDuration, now, v)) + res = append(res, counter.limiter.Reserve(ctx, gc.mainCfg.LTBMaxWaitDuration, now, v)) } } if d, err = WaitReservations(ctx, now, res); err == nil { diff --git a/client/resource_group/controller/util.go b/client/resource_group/controller/util.go new file mode 100644 index 00000000000..e3450e0ae0d --- /dev/null +++ b/client/resource_group/controller/util.go @@ -0,0 +1,68 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Copyright 2023 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS,g +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package controller + +import ( + "fmt" + "strconv" + "time" + + "github.com/pingcap/errors" +) + +// Duration is a wrapper of time.Duration for TOML and JSON. +type Duration struct { + time.Duration +} + +// NewDuration creates a Duration from time.Duration. +func NewDuration(duration time.Duration) Duration { + return Duration{Duration: duration} +} + +// MarshalJSON returns the duration as a JSON string. +func (d *Duration) MarshalJSON() ([]byte, error) { + return []byte(fmt.Sprintf(`"%s"`, d.String())), nil +} + +// UnmarshalJSON parses a JSON string into the duration. +func (d *Duration) UnmarshalJSON(text []byte) error { + s, err := strconv.Unquote(string(text)) + if err != nil { + return errors.WithStack(err) + } + duration, err := time.ParseDuration(s) + if err != nil { + return errors.WithStack(err) + } + d.Duration = duration + return nil +} + +// UnmarshalText parses a TOML string into the duration. +func (d *Duration) UnmarshalText(text []byte) error { + var err error + d.Duration, err = time.ParseDuration(string(text)) + return errors.WithStack(err) +} + +// MarshalText returns the duration as a JSON string. +func (d Duration) MarshalText() ([]byte, error) { + return []byte(d.String()), nil +} diff --git a/client/resource_group/controller/util_test.go b/client/resource_group/controller/util_test.go new file mode 100644 index 00000000000..b542e6713dc --- /dev/null +++ b/client/resource_group/controller/util_test.go @@ -0,0 +1,51 @@ +// Copyright 2023 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package controller + +import ( + "encoding/json" + "testing" + + "github.com/BurntSushi/toml" + "github.com/stretchr/testify/require" +) + +type example struct { + Interval Duration `json:"interval" toml:"interval"` +} + +func TestDurationJSON(t *testing.T) { + t.Parallel() + re := require.New(t) + example := &example{} + + text := []byte(`{"interval":"1h1m1s"}`) + re.NoError(json.Unmarshal(text, example)) + re.Equal(float64(60*60+60+1), example.Interval.Seconds()) + + b, err := json.Marshal(example) + re.NoError(err) + re.Equal(string(text), string(b)) +} + +func TestDurationTOML(t *testing.T) { + t.Parallel() + re := require.New(t) + example := &example{} + + text := []byte(`interval = "1h1m1s"`) + re.Nil(toml.Unmarshal(text, example)) + re.Equal(float64(60*60+60+1), example.Interval.Seconds()) +} diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index 61919a2ccb2..68b2de66ae2 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -31,9 +31,10 @@ import ( type actionType int const ( - add actionType = 0 - modify actionType = 1 - groupSettingsPathPrefix = "resource_group/settings" + add actionType = 0 + modify actionType = 1 + groupSettingsPathPrefix = "resource_group/settings" + controllerConfigPathPrefix = "resource_group/controller" // errNotPrimary is returned when the requested server is not primary. errNotPrimary = "not primary" // errNotLeader is returned when the requested server is not pd leader. @@ -43,6 +44,9 @@ const ( // GroupSettingsPathPrefixBytes is used to watch or get resource groups. var GroupSettingsPathPrefixBytes = []byte(groupSettingsPathPrefix) +// ControllerConfigPathPrefixBytes is used to watch or get controller config. +var ControllerConfigPathPrefixBytes = []byte(controllerConfigPathPrefix) + // ResourceManagerClient manages resource group info and token request. type ResourceManagerClient interface { ListResourceGroups(ctx context.Context) ([]*rmpb.ResourceGroup, error) diff --git a/pkg/mcs/resourcemanager/server/apis/v1/api.go b/pkg/mcs/resourcemanager/server/apis/v1/api.go index 7c5e3e010dc..970880788d4 100644 --- a/pkg/mcs/resourcemanager/server/apis/v1/api.go +++ b/pkg/mcs/resourcemanager/server/apis/v1/api.go @@ -16,7 +16,9 @@ package apis import ( "errors" + "fmt" "net/http" + "reflect" "sync" "github.com/gin-contrib/cors" @@ -29,6 +31,7 @@ import ( "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" + "github.com/tikv/pd/pkg/utils/reflectutil" ) // APIPathPrefix is the prefix of the API path. @@ -97,6 +100,8 @@ func (s *Service) RegisterRouter() { configEndpoint.GET("/group/:name", s.getResourceGroup) configEndpoint.GET("/groups", s.getResourceGroupList) configEndpoint.DELETE("/group/:name", s.deleteResourceGroup) + configEndpoint.GET("/controller", s.getControllerConfig) + configEndpoint.POST("/controller", s.setControllerConfig) } func (s *Service) handler() http.Handler { @@ -191,3 +196,43 @@ func (s *Service) deleteResourceGroup(c *gin.Context) { } c.String(http.StatusOK, "Success!") } + +// GetControllerConfig +// +// @Tags ResourceManager +// @Summary Get the resource controller config. +// @Success 200 {string} json format of rmserver.ControllerConfig +// @Failure 400 {string} error +// @Router /config/controller [GET] +func (s *Service) getControllerConfig(c *gin.Context) { + config := s.manager.GetControllerConfig() + c.IndentedJSON(http.StatusOK, config) +} + +// SetControllerConfig +// +// @Tags ResourceManager +// @Summary Set the resource controller config. +// @Param config body object true "json params, rmserver.ControllerConfig" +// @Success 200 {string} string "Success!" +// @Failure 400 {string} error +// @Router /config/controller [POST] +func (s *Service) setControllerConfig(c *gin.Context) { + conf := make(map[string]interface{}) + if err := c.ShouldBindJSON(&conf); err != nil { + c.String(http.StatusBadRequest, err.Error()) + return + } + for k, v := range conf { + key := reflectutil.FindJSONFullTagByChildTag(reflect.TypeOf(rmserver.ControllerConfig{}), k) + if key == "" { + c.String(http.StatusBadRequest, fmt.Sprintf("config item %s not found", k)) + return + } + if err := s.manager.UpdateControllerConfigItem(key, v); err != nil { + c.String(http.StatusBadRequest, err.Error()) + return + } + } + c.String(http.StatusOK, "Success!") +} diff --git a/pkg/mcs/resourcemanager/server/config.go b/pkg/mcs/resourcemanager/server/config.go index 51fbe388458..3f64b2987fd 100644 --- a/pkg/mcs/resourcemanager/server/config.go +++ b/pkg/mcs/resourcemanager/server/config.go @@ -57,6 +57,8 @@ const ( // Because the resource manager has not been deployed in microservice mode, // do not enable this function. defaultDegradedModeWaitDuration = time.Second * 0 + // defaultMaxWaitDuration is the max duration to wait for the token before throwing error. + defaultMaxWaitDuration = 30 * time.Second ) // Config is the configuration for the resource manager. @@ -94,6 +96,9 @@ type ControllerConfig struct { // EnableDegradedMode is to control whether resource control client enable degraded mode when server is disconnect. DegradedModeWaitDuration typeutil.Duration `toml:"degraded-mode-wait-duration" json:"degraded-mode-wait-duration"` + // LTBMaxWaitDuration is the max wait time duration for local token bucket. + LTBMaxWaitDuration typeutil.Duration `toml:"ltb-max-wait-duration" json:"ltb-max-wait-duration"` + // RequestUnit is the configuration determines the coefficients of the RRU and WRU cost. // This configuration should be modified carefully. RequestUnit RequestUnitConfig `toml:"request-unit" json:"request-unit"` @@ -107,6 +112,7 @@ func (rmc *ControllerConfig) Adjust(meta *configutil.ConfigMetaData) { rmc.RequestUnit.Adjust() configutil.AdjustDuration(&rmc.DegradedModeWaitDuration, defaultDegradedModeWaitDuration) + configutil.AdjustDuration(&rmc.LTBMaxWaitDuration, defaultMaxWaitDuration) failpoint.Inject("enableDegradedMode", func() { configutil.AdjustDuration(&rmc.DegradedModeWaitDuration, time.Second) }) diff --git a/pkg/mcs/resourcemanager/server/config_test.go b/pkg/mcs/resourcemanager/server/config_test.go index c0cac4da9c0..dd8dd2d2814 100644 --- a/pkg/mcs/resourcemanager/server/config_test.go +++ b/pkg/mcs/resourcemanager/server/config_test.go @@ -27,6 +27,7 @@ func TestControllerConfig(t *testing.T) { re := require.New(t) cfgData := ` [controller] +ltb-max-wait-duration = "60s" degraded-mode-wait-duration = "2s" [controller.request-unit] read-base-cost = 1.0 @@ -42,6 +43,7 @@ read-cpu-ms-cost = 5.0 re.NoError(err) re.Equal(cfg.Controller.DegradedModeWaitDuration.Duration, time.Second*2) + re.Equal(cfg.Controller.LTBMaxWaitDuration.Duration, time.Second*60) re.LessOrEqual(math.Abs(cfg.Controller.RequestUnit.CPUMsCost-5), 1e-7) re.LessOrEqual(math.Abs(cfg.Controller.RequestUnit.WriteCostPerByte-4), 1e-7) re.LessOrEqual(math.Abs(cfg.Controller.RequestUnit.WriteBaseCost-3), 1e-7) diff --git a/pkg/mcs/resourcemanager/server/manager.go b/pkg/mcs/resourcemanager/server/manager.go index 6d1b872575b..21866ee1156 100644 --- a/pkg/mcs/resourcemanager/server/manager.go +++ b/pkg/mcs/resourcemanager/server/manager.go @@ -19,10 +19,12 @@ import ( "encoding/json" "math" "sort" + "strings" "sync" "time" "github.com/gogo/protobuf/proto" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" @@ -30,6 +32,7 @@ import ( "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" + "github.com/tikv/pd/pkg/utils/jsonutil" "github.com/tikv/pd/pkg/utils/logutil" "go.uber.org/zap" ) @@ -102,32 +105,46 @@ func (m *Manager) GetBasicServer() bs.Server { // Init initializes the resource group manager. func (m *Manager) Init(ctx context.Context) error { - // Todo: If we can modify following configs in the future, we should reload these configs. - // Store the controller config into the storage. - m.storage.SaveControllerConfig(m.controllerConfig) + v, err := m.storage.LoadControllerConfig() + if err != nil { + log.Error("resource controller config load failed", zap.Error(err), zap.String("v", v)) + return err + } + if err = json.Unmarshal([]byte(v), &m.controllerConfig); err != nil { + log.Error("un-marshall controller config failed, fallback to default", zap.Error(err), zap.String("v", v)) + } + + // re-save the config to make sure the config has been persisted. + if err := m.storage.SaveControllerConfig(m.controllerConfig); err != nil { + return err + } // Load resource group meta info from storage. m.groups = make(map[string]*ResourceGroup) handler := func(k, v string) { group := &rmpb.ResourceGroup{} if err := proto.Unmarshal([]byte(v), group); err != nil { - log.Error("err", zap.Error(err), zap.String("k", k), zap.String("v", v)) + log.Error("failed to parse the resource group", zap.Error(err), zap.String("k", k), zap.String("v", v)) panic(err) } m.groups[group.Name] = FromProtoResourceGroup(group) } - m.storage.LoadResourceGroupSettings(handler) + if err := m.storage.LoadResourceGroupSettings(handler); err != nil { + return err + } // Load resource group states from storage. tokenHandler := func(k, v string) { tokens := &GroupStates{} if err := json.Unmarshal([]byte(v), tokens); err != nil { - log.Error("err", zap.Error(err), zap.String("k", k), zap.String("v", v)) + log.Error("failed to parse the resource group state", zap.Error(err), zap.String("k", k), zap.String("v", v)) panic(err) } if group, ok := m.groups[k]; ok { group.SetStatesIntoResourceGroup(tokens) } } - m.storage.LoadResourceGroupStates(tokenHandler) + if err := m.storage.LoadResourceGroupStates(tokenHandler); err != nil { + return err + } // Add default group if it's not inited. if _, ok := m.groups[reservedDefaultGroupName]; !ok { @@ -159,6 +176,47 @@ func (m *Manager) Init(ctx context.Context) error { return nil } +// UpdateControllerConfigItem updates the controller config item. +func (m *Manager) UpdateControllerConfigItem(key string, value interface{}) error { + kp := strings.Split(key, ".") + if len(kp) == 0 { + return errors.Errorf("invalid key %s", key) + } + m.Lock() + var config interface{} + switch kp[0] { + case "request-unit": + config = &m.controllerConfig.RequestUnit + default: + config = m.controllerConfig + } + updated, found, err := jsonutil.AddKeyValue(config, kp[len(kp)-1], value) + if err != nil { + m.Unlock() + return err + } + + if !found { + m.Unlock() + return errors.Errorf("config item %s not found", key) + } + m.Unlock() + if updated { + if err := m.storage.SaveControllerConfig(m.controllerConfig); err != nil { + log.Error("save controller config failed", zap.Error(err)) + } + log.Info("updated controller config item", zap.String("key", key), zap.Any("value", value)) + } + return nil +} + +// GetControllerConfig returns the controller config. +func (m *Manager) GetControllerConfig() *ControllerConfig { + m.RLock() + defer m.RUnlock() + return m.controllerConfig +} + // AddResourceGroup puts a resource group. // NOTE: AddResourceGroup should also be idempotent because tidb depends // on this retry mechanism. diff --git a/pkg/storage/endpoint/resource_group.go b/pkg/storage/endpoint/resource_group.go index f1b3feb36aa..150ea77a1c7 100644 --- a/pkg/storage/endpoint/resource_group.go +++ b/pkg/storage/endpoint/resource_group.go @@ -27,6 +27,7 @@ type ResourceGroupStorage interface { SaveResourceGroupStates(name string, obj interface{}) error DeleteResourceGroupStates(name string) error SaveControllerConfig(config interface{}) error + LoadControllerConfig() (string, error) } var _ ResourceGroupStorage = (*StorageEndpoint)(nil) @@ -65,3 +66,8 @@ func (se *StorageEndpoint) LoadResourceGroupStates(f func(k, v string)) error { func (se *StorageEndpoint) SaveControllerConfig(config interface{}) error { return se.saveJSON(controllerConfigPath, config) } + +// LoadControllerConfig loads the resource controller config from storage. +func (se *StorageEndpoint) LoadControllerConfig() (string, error) { + return se.Load(controllerConfigPath) +} diff --git a/server/server.go b/server/server.go index 7c19d8ff7c5..9e72477368d 100644 --- a/server/server.go +++ b/server/server.go @@ -1709,7 +1709,10 @@ func (s *Server) campaignLeader() { log.Info("triggering the leader callback functions") for _, cb := range s.leaderCallbacks { - cb(ctx) + if err := cb(ctx); err != nil { + log.Error("failed to execute leader callback function", errs.ZapError(err)) + return + } } // Try to create raft cluster. diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 926484cea1e..546339bee0f 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -19,6 +19,7 @@ import ( "encoding/json" "fmt" "io" + "math/rand" "net/http" "strconv" "strings" @@ -1265,3 +1266,106 @@ func (suite *resourceManagerClientTestSuite) TestSkipConsumptionForBackgroundJob c.Stop() } + +func (suite *resourceManagerClientTestSuite) TestResourceGroupControllerConfigChanged() { + re := suite.Require() + cli := suite.client + for _, group := range suite.initGroups { + resp, err := cli.AddResourceGroup(suite.ctx, group) + re.NoError(err) + re.Contains(resp, "Success!") + } + c1, err := controller.NewResourceGroupController(suite.ctx, 1, cli, nil) + re.NoError(err) + c1.Start(suite.ctx) + // with client option + c2, err := controller.NewResourceGroupController(suite.ctx, 2, cli, nil, controller.WithMaxWaitDuration(time.Hour)) + re.NoError(err) + c2.Start(suite.ctx) + // helper function for sending HTTP requests and checking responses + sendRequest := func(method, url string, body io.Reader) []byte { + req, err := http.NewRequest(method, url, body) + re.NoError(err) + resp, err := http.DefaultClient.Do(req) + re.NoError(err) + defer resp.Body.Close() + bytes, err := io.ReadAll(resp.Body) + re.NoError(err) + if resp.StatusCode != http.StatusOK { + re.Fail(string(bytes)) + } + return bytes + } + + getAddr := func() string { + server := suite.cluster.GetServer(suite.cluster.GetLeader()) + if rand.Intn(100)%2 == 1 { + server = suite.cluster.GetServer(suite.cluster.GetFollower()) + } + return server.GetAddr() + } + + configURL := "/resource-manager/api/v1/config/controller" + waitDuration := 10 * time.Second + readBaseCost := 1.5 + defaultCfg := controller.DefaultConfig() + // failpoint enableDegradedMode will setup and set it be 1s. + defaultCfg.DegradedModeWaitDuration.Duration = time.Second + expectRUCfg := controller.GenerateRUConfig(defaultCfg) + // initial config verification + respString := sendRequest("GET", getAddr()+configURL, nil) + defaultString, err := json.Marshal(defaultCfg) + re.NoError(err) + re.JSONEq(string(respString), string(defaultString)) + re.EqualValues(expectRUCfg, c1.GetConfig()) + + testCases := []struct { + configJSON string + value interface{} + expected func(ruConfig *controller.RUConfig) + }{ + { + configJSON: fmt.Sprintf(`{"degraded-mode-wait-duration": "%v"}`, waitDuration), + value: waitDuration, + expected: func(ruConfig *controller.RUConfig) { ruConfig.DegradedModeWaitDuration = waitDuration }, + }, + { + configJSON: fmt.Sprintf(`{"ltb-max-wait-duration": "%v"}`, waitDuration), + value: waitDuration, + expected: func(ruConfig *controller.RUConfig) { ruConfig.LTBMaxWaitDuration = waitDuration }, + }, + { + configJSON: fmt.Sprintf(`{"read-base-cost": %v}`, readBaseCost), + value: readBaseCost, + expected: func(ruConfig *controller.RUConfig) { ruConfig.ReadBaseCost = controller.RequestUnit(readBaseCost) }, + }, + { + configJSON: fmt.Sprintf(`{"write-base-cost": %v}`, readBaseCost*2), + value: readBaseCost * 2, + expected: func(ruConfig *controller.RUConfig) { ruConfig.WriteBaseCost = controller.RequestUnit(readBaseCost * 2) }, + }, + { + // reset the degraded-mode-wait-duration to default in test. + configJSON: fmt.Sprintf(`{"degraded-mode-wait-duration": "%v"}`, time.Second), + value: time.Second, + expected: func(ruConfig *controller.RUConfig) { ruConfig.DegradedModeWaitDuration = time.Second }, + }, + } + // change properties one by one and verify each time + for _, t := range testCases { + sendRequest("POST", getAddr()+configURL, strings.NewReader(t.configJSON)) + time.Sleep(500 * time.Millisecond) + t.expected(expectRUCfg) + re.EqualValues(expectRUCfg, c1.GetConfig()) + + expectRUCfg2 := *expectRUCfg + // always apply the client option + expectRUCfg2.LTBMaxWaitDuration = time.Hour + re.EqualValues(&expectRUCfg2, c2.GetConfig()) + } + // restart c1 + c1.Stop() + c1, err = controller.NewResourceGroupController(suite.ctx, 1, cli, nil) + re.NoError(err) + re.EqualValues(expectRUCfg, c1.GetConfig()) +} From 559ea7f247118eec9290d37793a217b116e73ddb Mon Sep 17 00:00:00 2001 From: buffer <1045931706@qq.com> Date: Tue, 19 Sep 2023 05:45:10 +0800 Subject: [PATCH 10/10] config: enable store limit v2 in raftstore-v2 (#7098) close tikv/pd#7099 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- server/cluster/cluster.go | 2 +- server/cluster/cluster_test.go | 4 ++++ server/config/persist_options.go | 6 ++++++ 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 29a8709bdac..1b2bc75e334 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -436,7 +436,7 @@ func (c *RaftCluster) runStoreConfigSync() { for { synced, switchRaftV2Config = c.syncStoreConfig(stores) if switchRaftV2Config { - if err := c.opt.Persist(c.GetStorage()); err != nil { + if err := c.opt.SwitchRaftV2(c.GetStorage()); err != nil { log.Warn("store config persisted failed", zap.Error(err)) } } diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index ea8d27b155f..5679fd6128d 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -1469,6 +1469,10 @@ func TestStoreConfigSync(t *testing.T) { err = opt.Reload(tc.GetStorage()) re.NoError(err) re.Equal(tc.GetOpts().(*config.PersistOptions).GetStoreConfig(), opt.GetStoreConfig()) + + re.Equal("v1", opt.GetScheduleConfig().StoreLimitVersion) + re.NoError(opt.SwitchRaftV2(tc.GetStorage())) + re.Equal("v2", opt.GetScheduleConfig().StoreLimitVersion) } func TestUpdateStorePendingPeerCount(t *testing.T) { diff --git a/server/config/persist_options.go b/server/config/persist_options.go index 1ea0b79424f..3f1c4d4a24e 100644 --- a/server/config/persist_options.go +++ b/server/config/persist_options.go @@ -762,6 +762,12 @@ type persistedConfig struct { StoreConfig sc.StoreConfig `json:"store"` } +// SwitchRaftV2 update some config if tikv raft engine switch into partition raft v2 +func (o *PersistOptions) SwitchRaftV2(storage endpoint.ConfigStorage) error { + o.GetScheduleConfig().StoreLimitVersion = "v2" + return o.Persist(storage) +} + // Persist saves the configuration to the storage. func (o *PersistOptions) Persist(storage endpoint.ConfigStorage) error { cfg := &persistedConfig{