diff --git a/client/client.go b/client/client.go index d5cf7cf28d00..ba4741e4ed9c 100644 --- a/client/client.go +++ b/client/client.go @@ -797,6 +797,12 @@ func (c *client) UpdateOption(option DynamicOption, value any) error { return errors.New("[pd] invalid value type for EnableFollowerHandle option, it should be bool") } c.option.setEnableFollowerHandle(enable) + case TSOClientRPCConcurrency: + value, ok := value.(int) + if !ok { + return errors.New("[pd] invalid value type for TSOClientRPCConcurrency option, it should be int") + } + c.option.setTSOClientRPCConcurrency(value) default: return errors.New("[pd] unsupported client option") } diff --git a/client/http/api.go b/client/http/api.go index d1bce99f4f9d..42d965ec17c4 100644 --- a/client/http/api.go +++ b/client/http/api.go @@ -60,7 +60,7 @@ const ( RegionLabelRulesByIDs = "/pd/api/v1/config/region-label/rules/ids" // Scheduler Schedulers = "/pd/api/v1/schedulers" - scatterRangeScheduler = "/pd/api/v1/schedulers/scatter-range-" + scatterRangeScheduler = "/pd/api/v1/schedulers/scatter-range-scheduler-" // Admin ResetTS = "/pd/api/v1/admin/reset-ts" BaseAllocID = "/pd/api/v1/admin/base-alloc-id" @@ -183,6 +183,7 @@ func SchedulerByName(name string) string { } // ScatterRangeSchedulerWithName returns the scatter range scheduler API with name parameter. +// It is used in https://github.com/pingcap/tidb/blob/2a3352c45dd0f8dd5102adb92879bbfa964e7f5f/pkg/server/handler/tikvhandler/tikv_handler.go#L1252. func ScatterRangeSchedulerWithName(name string) string { return fmt.Sprintf("%s%s", scatterRangeScheduler, name) } diff --git a/client/metrics.go b/client/metrics.go index a83b4a364076..d1b375aea8ac 100644 --- a/client/metrics.go +++ b/client/metrics.go @@ -47,6 +47,7 @@ var ( tsoBatchSendLatency prometheus.Histogram requestForwarded *prometheus.GaugeVec ongoingRequestCountGauge *prometheus.GaugeVec + estimateTSOLatencyGauge *prometheus.GaugeVec ) func initMetrics(constLabels prometheus.Labels) { @@ -127,6 +128,14 @@ func initMetrics(constLabels prometheus.Labels) { Help: "Current count of ongoing batch tso requests", ConstLabels: constLabels, }, []string{"stream"}) + estimateTSOLatencyGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "estimate_tso_latency", + Help: "Estimated latency of an RTT of getting TSO", + ConstLabels: constLabels, + }, []string{"stream"}) } var ( @@ -236,4 +245,5 @@ func registerMetrics() { prometheus.MustRegister(tsoBatchSize) prometheus.MustRegister(tsoBatchSendLatency) prometheus.MustRegister(requestForwarded) + prometheus.MustRegister(estimateTSOLatencyGauge) } diff --git a/client/option.go b/client/option.go index 0109bfc4ed09..3f2b7119b52f 100644 --- a/client/option.go +++ b/client/option.go @@ -29,6 +29,7 @@ const ( defaultMaxTSOBatchWaitInterval time.Duration = 0 defaultEnableTSOFollowerProxy = false defaultEnableFollowerHandle = false + defaultTSOClientRPCConcurrency = 1 ) // DynamicOption is used to distinguish the dynamic option type. @@ -43,6 +44,8 @@ const ( EnableTSOFollowerProxy // EnableFollowerHandle is the follower handle option. EnableFollowerHandle + // TSOClientRPCConcurrency controls the amount of ongoing TSO RPC requests at the same time in a single TSO client. + TSOClientRPCConcurrency dynamicOptionCount ) @@ -77,6 +80,7 @@ func newOption() *option { co.dynamicOptions[MaxTSOBatchWaitInterval].Store(defaultMaxTSOBatchWaitInterval) co.dynamicOptions[EnableTSOFollowerProxy].Store(defaultEnableTSOFollowerProxy) co.dynamicOptions[EnableFollowerHandle].Store(defaultEnableFollowerHandle) + co.dynamicOptions[TSOClientRPCConcurrency].Store(defaultTSOClientRPCConcurrency) return co } @@ -127,3 +131,14 @@ func (o *option) setEnableTSOFollowerProxy(enable bool) { func (o *option) getEnableTSOFollowerProxy() bool { return o.dynamicOptions[EnableTSOFollowerProxy].Load().(bool) } + +func (o *option) setTSOClientRPCConcurrency(value int) { + old := o.getTSOClientRPCConcurrency() + if value != old { + o.dynamicOptions[TSOClientRPCConcurrency].Store(value) + } +} + +func (o *option) getTSOClientRPCConcurrency() int { + return o.dynamicOptions[TSOClientRPCConcurrency].Load().(int) +} diff --git a/client/tso_batch_controller.go b/client/tso_batch_controller.go index 32191889160e..b810e1086674 100644 --- a/client/tso_batch_controller.go +++ b/client/tso_batch_controller.go @@ -64,6 +64,17 @@ func (tbc *tsoBatchController) fetchPendingRequests(ctx context.Context, tsoRequ // TODO: `tbc.collectedRequestCount` should never be non-empty here. Consider do assertion here. tbc.collectedRequestCount = 0 for { + // If the batch size reaches the maxBatchSize limit but the token haven't arrived yet, don't receive more + // requests, and return when token is ready. + if tbc.collectedRequestCount >= tbc.maxBatchSize && !tokenAcquired { + select { + case <-ctx.Done(): + return ctx.Err() + case <-tokenCh: + return nil + } + } + select { case <-ctx.Done(): return ctx.Err() @@ -146,6 +157,37 @@ fetchPendingRequestsLoop: return nil } +// fetchRequestsWithTimer tries to fetch requests until the given timer ticks. The caller must set the timer properly +// before calling this function. +func (tbc *tsoBatchController) fetchRequestsWithTimer(ctx context.Context, tsoRequestCh <-chan *tsoRequest, timer *time.Timer) error { +batchingLoop: + for tbc.collectedRequestCount < tbc.maxBatchSize { + select { + case <-ctx.Done(): + return ctx.Err() + case req := <-tsoRequestCh: + tbc.pushRequest(req) + case <-timer.C: + break batchingLoop + } + } + + // Try to collect more requests in non-blocking way. +nonWaitingBatchLoop: + for tbc.collectedRequestCount < tbc.maxBatchSize { + select { + case <-ctx.Done(): + return ctx.Err() + case req := <-tsoRequestCh: + tbc.pushRequest(req) + default: + break nonWaitingBatchLoop + } + } + + return nil +} + func (tbc *tsoBatchController) pushRequest(tsoReq *tsoRequest) { tbc.collectedRequests[tbc.collectedRequestCount] = tsoReq tbc.collectedRequestCount++ diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index a1e0b03a1fae..7febf194f3cb 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -17,9 +17,11 @@ package pd import ( "context" "fmt" + "math" "math/rand" "runtime/trace" "sync" + "sync/atomic" "time" "github.com/opentracing/opentracing-go" @@ -61,6 +63,7 @@ type tsoInfo struct { respReceivedAt time.Time physical int64 logical int64 + sourceStreamID string } type tsoServiceProvider interface { @@ -69,6 +72,8 @@ type tsoServiceProvider interface { updateConnectionCtxs(ctx context.Context, dc string, connectionCtxs *sync.Map) bool } +const dispatcherCheckRPCConcurrencyInterval = time.Second * 5 + type tsoDispatcher struct { ctx context.Context cancel context.CancelFunc @@ -79,7 +84,7 @@ type tsoDispatcher struct { connectionCtxs *sync.Map tsoRequestCh chan *tsoRequest tsDeadlineCh chan *deadline - lastTSOInfo *tsoInfo + latestTSOInfo atomic.Pointer[tsoInfo] // For reusing tsoBatchController objects batchBufferPool *sync.Pool @@ -87,7 +92,10 @@ type tsoDispatcher struct { // A token must be acquired here before sending an RPC request, and the token must be put back after finishing the // RPC. This is used like a semaphore, but we don't use semaphore directly here as it cannot be selected with // other channels. - tokenCh chan struct{} + tokenCh chan struct{} + lastCheckConcurrencyTime time.Time + tokenCount int + rpcConcurrency int updateConnectionCtxsCh chan struct{} } @@ -115,7 +123,7 @@ func newTSODispatcher( provider: provider, connectionCtxs: &sync.Map{}, tsoRequestCh: tsoRequestCh, - tsDeadlineCh: make(chan *deadline, 1), + tsDeadlineCh: make(chan *deadline, tokenChCapacity), batchBufferPool: &sync.Pool{ New: func() any { return newTSOBatchController(maxBatchSize * 2) @@ -187,9 +195,6 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { batchController *tsoBatchController ) - // Currently only 1 concurrency is supported. Put one token in. - td.tokenCh <- struct{}{} - log.Info("[tso] tso dispatcher created", zap.String("dc-location", dc)) // Clean up the connectionCtxs when the dispatcher exits. defer func() { @@ -200,6 +205,7 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { return true }) if batchController != nil && batchController.collectedRequestCount != 0 { + // If you encounter this failure, please check the stack in the logs to see if it's a panic. log.Fatal("batched tso requests not cleared when exiting the tso dispatcher loop", zap.Any("panic", recover())) } tsoErr := errors.WithStack(errClosing) @@ -219,6 +225,12 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { // Loop through each batch of TSO requests and send them for processing. streamLoopTimer := time.NewTimer(option.timeout) defer streamLoopTimer.Stop() + + // Create a not-started-timer to be used for collecting batches for concurrent RPC. + batchingTimer := time.NewTimer(0) + <-batchingTimer.C + defer batchingTimer.Stop() + bo := retry.InitialBackoffer(updateMemberBackOffBaseTime, updateMemberTimeout, updateMemberBackOffBaseTime) tsoBatchLoop: for { @@ -233,8 +245,18 @@ tsoBatchLoop: batchController = td.batchBufferPool.Get().(*tsoBatchController) } - // Start to collect the TSO requests. maxBatchWaitInterval := option.getMaxTSOBatchWaitInterval() + + currentBatchStartTime := time.Now() + // Update concurrency settings if needed. + if err = td.checkTSORPCConcurrency(ctx, maxBatchWaitInterval, currentBatchStartTime); err != nil { + // checkTSORPCConcurrency can only fail due to `ctx` being invalidated. + log.Info("[tso] stop checking tso rpc concurrency configurations due to context canceled", + zap.String("dc-location", dc), zap.Error(err)) + return + } + + // Start to collect the TSO requests. // Once the TSO requests are collected, must make sure they could be finished or revoked eventually, // otherwise the upper caller may get blocked on waiting for the results. if err = batchController.fetchPendingRequests(ctx, td.tsoRequestCh, td.tokenCh, maxBatchWaitInterval); err != nil { @@ -318,6 +340,57 @@ tsoBatchLoop: break streamChoosingLoop } + + noDelay := false + failpoint.Inject("tsoDispatcherConcurrentModeNoDelay", func() { + noDelay = true + }) + + // If concurrent RPC is enabled, the time for collecting each request batch is expected to be + // estimatedRPCDuration / concurrency. Note the time mentioned here is counted from starting trying to collect + // the batch, instead of the time when the first request arrives. + // Here, if the elapsed time since starting collecting this batch didn't reach the expected batch time, then + // continue collecting. + if td.isConcurrentRPCEnabled() { + estimatedLatency := stream.EstimatedRPCLatency() + goalBatchTime := estimatedLatency / time.Duration(td.rpcConcurrency) + + failpoint.Inject("tsoDispatcherConcurrentModeAssertDelayDuration", func(val failpoint.Value) { + if s, ok := val.(string); ok { + expected, err := time.ParseDuration(s) + if err != nil { + panic(err) + } + if math.Abs(expected.Seconds()-goalBatchTime.Seconds()) > 1e-6 { + log.Fatal("tsoDispatcher: trying to delay for unexpected duration for the batch", zap.Duration("goalBatchTime", goalBatchTime), zap.Duration("expectedBatchTime", expected)) + } + } else { + panic("invalid value for failpoint tsoDispatcherConcurrentModeAssertDelayDuration: expected string") + } + }) + + waitTimerStart := time.Now() + remainingBatchTime := goalBatchTime - waitTimerStart.Sub(currentBatchStartTime) + if remainingBatchTime > 0 && !noDelay { + if !batchingTimer.Stop() { + select { + case <-batchingTimer.C: + default: + } + } + batchingTimer.Reset(remainingBatchTime) + + err = batchController.fetchRequestsWithTimer(ctx, td.tsoRequestCh, batchingTimer) + if err != nil { + // There should not be other kinds of errors. + log.Info("[tso] stop fetching the pending tso requests due to context canceled", + zap.String("dc-location", dc), zap.Error(err)) + td.cancelCollectedRequests(batchController, invalidStreamID, errors.WithStack(ctx.Err())) + return + } + } + } + done := make(chan struct{}) dl := newTSDeadline(option.timeout, done, cancel) select { @@ -495,6 +568,9 @@ func (td *tsoDispatcher) processRequests( reqKeyspaceGroupID = svcDiscovery.GetKeyspaceGroupID() ) + // Load latest allocated ts for monotonicity assertion. + tsoInfoBeforeReq := td.latestTSOInfo.Load() + cb := func(result tsoRequestResult, reqKeyspaceGroupID uint32, err error) { // As golang doesn't allow double-closing a channel, here is implicitly a check that the callback // is never called twice or called while it's also being cancelled elsewhere. @@ -513,10 +589,12 @@ func (td *tsoDispatcher) processRequests( respReceivedAt: time.Now(), physical: result.physical, logical: result.logical, + sourceStreamID: stream.streamID, } // `logical` is the largest ts's logical part here, we need to do the subtracting before we finish each TSO request. firstLogical := tsoutil.AddLogical(result.logical, -int64(result.count)+1, result.suffixBits) - td.compareAndSwapTS(curTSOInfo, firstLogical) + // Do the check before releasing the token. + td.checkMonotonicity(tsoInfoBeforeReq, curTSOInfo, firstLogical) td.doneCollectedRequests(tbc, result.physical, firstLogical, result.suffixBits, stream.streamID) } @@ -542,32 +620,35 @@ func (td *tsoDispatcher) doneCollectedRequests(tbc *tsoBatchController, physical tbc.finishCollectedRequests(physical, firstLogical, suffixBits, streamID, nil) } -func (td *tsoDispatcher) compareAndSwapTS( - curTSOInfo *tsoInfo, firstLogical int64, +// checkMonotonicity checks whether the monotonicity of the TSO allocation is violated. +// It asserts (curTSOInfo, firstLogical) must be larger than lastTSOInfo, and updates td.latestTSOInfo if it grows. +// +// Note that when concurrent RPC is enabled, the lastTSOInfo may not be the latest value stored in td.latestTSOInfo +// field. Instead, it's the value that was loaded just before the current RPC request's beginning. The reason is, +// if two requests processing time has overlap, they don't have a strong order, and the later-finished one may be +// allocated later (with larger value) than another. We only need to guarantee request A returns larger ts than B +// if request A *starts* after request B *finishes*. +func (td *tsoDispatcher) checkMonotonicity( + lastTSOInfo *tsoInfo, curTSOInfo *tsoInfo, firstLogical int64, ) { - if td.lastTSOInfo != nil { - var ( - lastTSOInfo = td.lastTSOInfo - dc = td.dc - physical = curTSOInfo.physical - keyspaceID = td.provider.getServiceDiscovery().GetKeyspaceID() - ) - if td.lastTSOInfo.respKeyspaceGroupID != curTSOInfo.respKeyspaceGroupID { + keyspaceID := td.provider.getServiceDiscovery().GetKeyspaceID() + if lastTSOInfo != nil { + if lastTSOInfo.respKeyspaceGroupID != curTSOInfo.respKeyspaceGroupID { log.Info("[tso] keyspace group changed", - zap.String("dc-location", dc), + zap.String("dc-location", td.dc), zap.Uint32("old-group-id", lastTSOInfo.respKeyspaceGroupID), zap.Uint32("new-group-id", curTSOInfo.respKeyspaceGroupID)) } // The TSO we get is a range like [largestLogical-count+1, largestLogical], so we save the last TSO's largest logical // to compare with the new TSO's first logical. For example, if we have a TSO resp with logical 10, count 5, then - // all TSOs we get will be [6, 7, 8, 9, 10]. lastTSOInfo.logical stores the logical part of the largest ts returned + // all TSOs we get will be [6, 7, 8, 9, 10]. latestTSOInfo.logical stores the logical part of the largest ts returned // last time. - if tsoutil.TSLessEqual(physical, firstLogical, lastTSOInfo.physical, lastTSOInfo.logical) { + if tsoutil.TSLessEqual(curTSOInfo.physical, firstLogical, lastTSOInfo.physical, lastTSOInfo.logical) { log.Panic("[tso] timestamp fallback", - zap.String("dc-location", dc), + zap.String("dc-location", td.dc), zap.Uint32("keyspace", keyspaceID), zap.String("last-ts", fmt.Sprintf("(%d, %d)", lastTSOInfo.physical, lastTSOInfo.logical)), - zap.String("cur-ts", fmt.Sprintf("(%d, %d)", physical, firstLogical)), + zap.String("cur-ts", fmt.Sprintf("(%d, %d)", curTSOInfo.physical, firstLogical)), zap.String("last-tso-server", lastTSOInfo.tsoServer), zap.String("cur-tso-server", curTSOInfo.tsoServer), zap.Uint32("last-keyspace-group-in-request", lastTSOInfo.reqKeyspaceGroupID), @@ -575,8 +656,103 @@ func (td *tsoDispatcher) compareAndSwapTS( zap.Uint32("last-keyspace-group-in-response", lastTSOInfo.respKeyspaceGroupID), zap.Uint32("cur-keyspace-group-in-response", curTSOInfo.respKeyspaceGroupID), zap.Time("last-response-received-at", lastTSOInfo.respReceivedAt), - zap.Time("cur-response-received-at", curTSOInfo.respReceivedAt)) + zap.Time("cur-response-received-at", curTSOInfo.respReceivedAt), + zap.String("last-stream-id", lastTSOInfo.sourceStreamID), + zap.String("cur-stream-id", curTSOInfo.sourceStreamID)) + } + } + + if td.latestTSOInfo.CompareAndSwap(nil, curTSOInfo) { + // If latestTSOInfo is missing, simply store it and exit. + return + } + + // Replace if we are holding a larger ts than that has been recorded. + for { + old := td.latestTSOInfo.Load() + if tsoutil.TSLessEqual(curTSOInfo.physical, curTSOInfo.logical, old.physical, old.logical) { + // The current one is large enough. Skip. + break + } + if td.latestTSOInfo.CompareAndSwap(old, curTSOInfo) { + // Successfully replaced. + break + } + } +} + +// checkTSORPCConcurrency checks configurations about TSO RPC concurrency, and adjust the token count if needed. +// Some other options (EnableTSOFollowerProxy and MaxTSOBatchWaitInterval) may affect the availability of concurrent +// RPC requests. As the dispatcher loop loads MaxTSOBatchWaitInterval in each single circle, pass it directly to this +// function. Other configurations will be loaded within this function when needed. +// +// Behavior of the function: +// - As concurrent TSO RPC requests is an optimization aiming on the opposite purpose to that of EnableTSOFollowerProxy +// and MaxTSOBatchWaitInterval, so once either EnableTSOFollowerProxy and MaxTSOBatchWaitInterval is enabled, the +// concurrency will always be set to 1 no matter how the user configured it. +// - Normally, this function takes effect in a limited frequency controlled by dispatcherCheckRPCConcurrencyInterval. +// However, if the RPC concurrency is set to more than 1, and MaxTSOBatchWaitInterval is changed from disabled into +// enabled (0 -> positive), this function takes effect immediately to disable concurrent RPC requests. +// - After this function takes effect, the final decision of concurrency and token count will be set to +// td.rpcConcurrency and td.tokenCount; and tokens available in td.tokenCh will also be adjusted. +func (td *tsoDispatcher) checkTSORPCConcurrency(ctx context.Context, maxBatchWaitInterval time.Duration, now time.Time) error { + // If we currently enabled concurrent TSO RPC requests, but `maxBatchWaitInterval` is a positive value, it must + // because that MaxTSOBatchWaitInterval is just enabled. In this case, disable concurrent TSO RPC requests + // immediately, because MaxTSOBatchWaitInterval and concurrent RPC requests has opposite purpose. + immediatelyUpdate := td.rpcConcurrency > 1 && maxBatchWaitInterval > 0 + + // Allow always updating for test purpose. + failpoint.Inject("tsoDispatcherAlwaysCheckConcurrency", func() { + immediatelyUpdate = true + }) + + if !immediatelyUpdate && now.Sub(td.lastCheckConcurrencyTime) < dispatcherCheckRPCConcurrencyInterval { + return nil + } + td.lastCheckConcurrencyTime = now + + newConcurrency := td.provider.getOption().getTSOClientRPCConcurrency() + if maxBatchWaitInterval > 0 || td.provider.getOption().getEnableTSOFollowerProxy() { + newConcurrency = 1 + } + + if newConcurrency == td.rpcConcurrency { + return nil + } + + log.Info("[tso] switching tso rpc concurrency", zap.Int("old", td.rpcConcurrency), zap.Int("new", newConcurrency)) + td.rpcConcurrency = newConcurrency + + // Find a proper token count. + // When the concurrency is set to 1, there's only 1 token, which means only 1 RPC request can run at the same + // time. + // When the concurrency is set to more than 1, the time interval between sending two batches of requests is + // controlled by an estimation of an average RPC duration. But as the duration of an RPC may jitter in the network, + // and an RPC request may finish earlier or later. So we allow there to be the actual number of concurrent ongoing + // request to be fluctuating. So in this case, the token count will be set to 2 times the expected concurrency. + newTokenCount := newConcurrency + if newConcurrency > 1 { + newTokenCount = newConcurrency * 2 + } + + if newTokenCount > td.tokenCount { + for td.tokenCount < newTokenCount { + td.tokenCh <- struct{}{} + td.tokenCount++ + } + } else if newTokenCount < td.tokenCount { + for td.tokenCount > newTokenCount { + select { + case <-ctx.Done(): + return ctx.Err() + case <-td.tokenCh: + } + td.tokenCount-- } } - td.lastTSOInfo = curTSOInfo + return nil +} + +func (td *tsoDispatcher) isConcurrentRPCEnabled() bool { + return td.rpcConcurrency > 1 } diff --git a/client/tso_dispatcher_test.go b/client/tso_dispatcher_test.go index b8f0fcef2086..bf038e7b7f37 100644 --- a/client/tso_dispatcher_test.go +++ b/client/tso_dispatcher_test.go @@ -18,20 +18,27 @@ import ( "context" "fmt" "sync" + "sync/atomic" "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" "go.uber.org/zap/zapcore" ) type mockTSOServiceProvider struct { - option *option + option *option + createStream func(ctx context.Context) *tsoStream + updateConnMu sync.Mutex } -func newMockTSOServiceProvider(option *option) *mockTSOServiceProvider { +func newMockTSOServiceProvider(option *option, createStream func(ctx context.Context) *tsoStream) *mockTSOServiceProvider { return &mockTSOServiceProvider{ - option: option, + option: option, + createStream: createStream, } } @@ -43,17 +50,279 @@ func (*mockTSOServiceProvider) getServiceDiscovery() ServiceDiscovery { return NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) } -func (*mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, _dc string, connectionCtxs *sync.Map) bool { +func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, _dc string, connectionCtxs *sync.Map) bool { + // Avoid concurrent updating in the background updating goroutine and active updating in the dispatcher loop when + // stream is missing. + m.updateConnMu.Lock() + defer m.updateConnMu.Unlock() + _, ok := connectionCtxs.Load(mockStreamURL) if ok { return true } ctx, cancel := context.WithCancel(ctx) - stream := newTSOStream(ctx, mockStreamURL, newMockTSOStreamImpl(ctx, true)) + var stream *tsoStream + if m.createStream == nil { + stream = newTSOStream(ctx, mockStreamURL, newMockTSOStreamImpl(ctx, resultModeGenerated)) + } else { + stream = m.createStream(ctx) + } connectionCtxs.LoadOrStore(mockStreamURL, &tsoConnectionContext{ctx, cancel, mockStreamURL, stream}) return true } +type testTSODispatcherSuite struct { + suite.Suite + re *require.Assertions + + streamInner *mockTSOStreamImpl + stream *tsoStream + dispatcher *tsoDispatcher + dispatcherWg sync.WaitGroup + option *option + + reqPool *sync.Pool +} + +func (s *testTSODispatcherSuite) SetupTest() { + s.re = require.New(s.T()) + s.option = newOption() + s.option.timeout = time.Hour + // As the internal logic of the tsoDispatcher allows it to create streams multiple times, but our tests needs + // single stable access to the inner stream, we do not allow it to create it more than once in these tests. + creating := new(atomic.Bool) + // To avoid data race on reading `stream` and `streamInner` fields. + created := new(atomic.Bool) + createStream := func(ctx context.Context) *tsoStream { + if !creating.CompareAndSwap(false, true) { + s.re.FailNow("testTSODispatcherSuite: trying to create stream more than once, which is unsupported in this tests") + } + s.streamInner = newMockTSOStreamImpl(ctx, resultModeGenerateOnSignal) + s.stream = newTSOStream(ctx, mockStreamURL, s.streamInner) + created.Store(true) + return s.stream + } + s.dispatcher = newTSODispatcher(context.Background(), globalDCLocation, defaultMaxTSOBatchSize, newMockTSOServiceProvider(s.option, createStream)) + s.reqPool = &sync.Pool{ + New: func() any { + return &tsoRequest{ + done: make(chan error, 1), + physical: 0, + logical: 0, + dcLocation: globalDCLocation, + } + }, + } + + s.dispatcherWg.Add(1) + go s.dispatcher.handleDispatcher(&s.dispatcherWg) + + // Perform a request to ensure the stream must be created. + + { + ctx := context.Background() + req := s.sendReq(ctx) + s.reqMustNotReady(req) + // Wait until created + for !created.Load() { + time.Sleep(time.Millisecond) + } + s.streamInner.generateNext() + s.reqMustReady(req) + } + s.re.True(created.Load()) + s.re.NotNil(s.stream) +} + +func (s *testTSODispatcherSuite) TearDownTest() { + s.dispatcher.close() + s.streamInner.stop() + s.dispatcherWg.Wait() + s.stream.WaitForClosed() + s.streamInner = nil + s.stream = nil + s.dispatcher = nil + s.reqPool = nil +} + +func (s *testTSODispatcherSuite) getReq(ctx context.Context) *tsoRequest { + req := s.reqPool.Get().(*tsoRequest) + req.clientCtx = context.Background() + req.requestCtx = ctx + req.physical = 0 + req.logical = 0 + req.start = time.Now() + req.pool = s.reqPool + return req +} + +func (s *testTSODispatcherSuite) sendReq(ctx context.Context) *tsoRequest { + req := s.getReq(ctx) + s.dispatcher.push(req) + return req +} + +func (s *testTSODispatcherSuite) reqMustNotReady(req *tsoRequest) { + _, _, err := req.waitTimeout(time.Millisecond * 50) + s.re.Error(err) + s.re.ErrorIs(err, context.DeadlineExceeded) +} + +func (s *testTSODispatcherSuite) reqMustReady(req *tsoRequest) (physical int64, logical int64) { + physical, logical, err := req.waitTimeout(time.Second) + s.re.NoError(err) + return physical, logical +} + +func TestTSODispatcherTestSuite(t *testing.T) { + suite.Run(t, new(testTSODispatcherSuite)) +} + +func (s *testTSODispatcherSuite) TestBasic() { + ctx := context.Background() + req := s.sendReq(ctx) + s.reqMustNotReady(req) + s.streamInner.generateNext() + s.reqMustReady(req) +} + +func (s *testTSODispatcherSuite) checkIdleTokenCount(expectedTotal int) { + // When the tsoDispatcher is idle, the dispatcher loop will acquire a token and wait for requests. Therefore + // there should be N-1 free tokens remaining. + spinStart := time.Now() + for time.Since(spinStart) < time.Second { + if s.dispatcher.tokenCount != expectedTotal { + continue + } + if len(s.dispatcher.tokenCh) == expectedTotal-1 { + break + } + } + s.re.Equal(expectedTotal, s.dispatcher.tokenCount) + s.re.Len(s.dispatcher.tokenCh, expectedTotal-1) +} + +func (s *testTSODispatcherSuite) testStaticConcurrencyImpl(concurrency int) { + ctx := context.Background() + s.option.setTSOClientRPCConcurrency(concurrency) + + // Make sure the state of the mock stream is clear. Unexpected batching may make the requests sent to the stream + // less than expected, causing there are more `generateNext` signals or generated results. + s.re.Empty(s.streamInner.resultCh) + + // The dispatcher may block on fetching requests, which is after checking concurrency option. Perform a request + // to make sure the concurrency setting takes effect. + req := s.sendReq(ctx) + s.reqMustNotReady(req) + s.streamInner.generateNext() + s.reqMustReady(req) + + // For concurrent mode, the actual token count is twice the concurrency. + // Note that the concurrency is a hint, and it's allowed to have more than `concurrency` requests running. + tokenCount := concurrency + if concurrency > 1 { + tokenCount = concurrency * 2 + } + s.checkIdleTokenCount(tokenCount) + + // As the failpoint `tsoDispatcherConcurrentModeNoDelay` is set, tsoDispatcher won't collect requests in blocking + // way. And as `reqMustNotReady` delays for a while, requests shouldn't be batched as long as there are free tokens. + // The first N requests (N=tokenCount) will each be a single batch, occupying a token. The last 3 are blocked, + // and will be batched together once there is a free token. + reqs := make([]*tsoRequest, 0, tokenCount+3) + + for i := 0; i < tokenCount+3; i++ { + req := s.sendReq(ctx) + s.reqMustNotReady(req) + reqs = append(reqs, req) + } + + // The dispatcher won't process more request batches if tokens are used up. + // Note that `reqMustNotReady` contains a delay, which makes it nearly impossible that dispatcher is processing the + // second batch but not finished yet. + // Also note that in current implementation, the tsoStream tries to receive the next result before checking + // the `tsoStream.pendingRequests` queue. Changing this behavior may need to update this test. + for i := 0; i < tokenCount+3; i++ { + expectedPending := tokenCount + 1 - i + if expectedPending > tokenCount { + expectedPending = tokenCount + } + if expectedPending < 0 { + expectedPending = 0 + } + + // Spin for a while as the dispatcher loop may have not finished sending next batch to pendingRequests + spinStart := time.Now() + for time.Since(spinStart) < time.Second { + if expectedPending == len(s.stream.pendingRequests) { + break + } + } + s.re.Len(s.stream.pendingRequests, expectedPending) + + req := reqs[i] + // The last 3 requests should be in a single batch. Don't need to generate new results for the last 2. + if i <= tokenCount { + s.reqMustNotReady(req) + s.streamInner.generateNext() + } + s.reqMustReady(req) + } +} + +func (s *testTSODispatcherSuite) TestConcurrentRPC() { + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeNoDelay", "return")) + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherAlwaysCheckConcurrency", "return")) + defer func() { + s.re.NoError(failpoint.Disable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeNoDelay")) + s.re.NoError(failpoint.Disable("github.com/tikv/pd/client/tsoDispatcherAlwaysCheckConcurrency")) + }() + + s.testStaticConcurrencyImpl(1) + s.testStaticConcurrencyImpl(2) + s.testStaticConcurrencyImpl(4) + s.testStaticConcurrencyImpl(16) +} + +func (s *testTSODispatcherSuite) TestBatchDelaying() { + ctx := context.Background() + s.option.setTSOClientRPCConcurrency(2) + + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeNoDelay", "return")) + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoStreamSimulateEstimatedRPCLatency", `return("12ms")`)) + defer func() { + s.re.NoError(failpoint.Disable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeNoDelay")) + s.re.NoError(failpoint.Disable("github.com/tikv/pd/client/tsoStreamSimulateEstimatedRPCLatency")) + }() + + // Make sure concurrency option takes effect. + req := s.sendReq(ctx) + s.streamInner.generateNext() + s.reqMustReady(req) + + // Trigger the check. + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeAssertDelayDuration", `return("6ms")`)) + defer func() { + s.re.NoError(failpoint.Disable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeAssertDelayDuration")) + }() + req = s.sendReq(ctx) + s.streamInner.generateNext() + s.reqMustReady(req) + + // Try other concurrency. + s.option.setTSOClientRPCConcurrency(3) + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeAssertDelayDuration", `return("4ms")`)) + req = s.sendReq(ctx) + s.streamInner.generateNext() + s.reqMustReady(req) + + s.option.setTSOClientRPCConcurrency(4) + s.re.NoError(failpoint.Enable("github.com/tikv/pd/client/tsoDispatcherConcurrentModeAssertDelayDuration", `return("3ms")`)) + req = s.sendReq(ctx) + s.streamInner.generateNext() + s.reqMustReady(req) +} + func BenchmarkTSODispatcherHandleRequests(b *testing.B) { log.SetLevel(zapcore.FatalLevel) @@ -80,7 +349,7 @@ func BenchmarkTSODispatcherHandleRequests(b *testing.B) { return req } - dispatcher := newTSODispatcher(ctx, globalDCLocation, defaultMaxTSOBatchSize, newMockTSOServiceProvider(newOption())) + dispatcher := newTSODispatcher(ctx, globalDCLocation, defaultMaxTSOBatchSize, newMockTSOServiceProvider(newOption(), nil)) var wg sync.WaitGroup wg.Add(1) diff --git a/client/tso_request.go b/client/tso_request.go index fb2ae2bb92e5..5c959673a8b5 100644 --- a/client/tso_request.go +++ b/client/tso_request.go @@ -60,6 +60,11 @@ func (req *tsoRequest) tryDone(err error) { // Wait will block until the TSO result is ready. func (req *tsoRequest) Wait() (physical int64, logical int64, err error) { + return req.waitCtx(req.requestCtx) +} + +// waitCtx waits for the TSO result with specified ctx, while not using req.requestCtx. +func (req *tsoRequest) waitCtx(ctx context.Context) (physical int64, logical int64, err error) { // If tso command duration is observed very high, the reason could be it // takes too long for Wait() be called. start := time.Now() @@ -78,13 +83,20 @@ func (req *tsoRequest) Wait() (physical int64, logical int64, err error) { cmdDurationWait.Observe(now.Sub(start).Seconds()) cmdDurationTSO.Observe(now.Sub(req.start).Seconds()) return - case <-req.requestCtx.Done(): - return 0, 0, errors.WithStack(req.requestCtx.Err()) + case <-ctx.Done(): + return 0, 0, errors.WithStack(ctx.Err()) case <-req.clientCtx.Done(): return 0, 0, errors.WithStack(req.clientCtx.Err()) } } +// waitTimeout waits for the TSO result for limited time. Currently only for test purposes. +func (req *tsoRequest) waitTimeout(timeout time.Duration) (physical int64, logical int64, err error) { + ctx, cancel := context.WithTimeout(req.requestCtx, timeout) + defer cancel() + return req.waitCtx(ctx) +} + type tsoRequestFastFail struct { err error } diff --git a/client/tso_stream.go b/client/tso_stream.go index 479beff2c6ad..142ad71c6b99 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -18,11 +18,13 @@ import ( "context" "fmt" "io" + "math" "sync" "sync/atomic" "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" @@ -214,6 +216,8 @@ type tsoStream struct { state atomic.Int32 stoppedWithErr atomic.Pointer[error] + estimatedLatencyMicros atomic.Uint64 + ongoingRequestCountGauge prometheus.Gauge ongoingRequests atomic.Int32 } @@ -226,7 +230,10 @@ const ( var streamIDAlloc atomic.Int32 -const invalidStreamID = "" +const ( + invalidStreamID = "" + maxPendingRequestsInTSOStream = 64 +) func newTSOStream(ctx context.Context, serverURL string, stream grpcTSOStreamAdapter) *tsoStream { streamID := fmt.Sprintf("%s-%d", serverURL, streamIDAlloc.Add(1)) @@ -238,7 +245,7 @@ func newTSOStream(ctx context.Context, serverURL string, stream grpcTSOStreamAda stream: stream, streamID: streamID, - pendingRequests: make(chan batchedRequests, 64), + pendingRequests: make(chan batchedRequests, maxPendingRequestsInTSOStream), cancel: cancel, @@ -363,6 +370,27 @@ func (s *tsoStream) recvLoop(ctx context.Context) { s.ongoingRequestCountGauge.Set(0) }() + // For calculating the estimated RPC latency. + const ( + filterCutoffFreq float64 = 1.0 + filterNewSampleWeightUpperbound float64 = 0.2 + ) + // The filter applies on logarithm of the latency of each TSO RPC in microseconds. + filter := newRCFilter(filterCutoffFreq, filterNewSampleWeightUpperbound) + + updateEstimatedLatency := func(sampleTime time.Time, latency time.Duration) { + if latency < 0 { + // Unreachable + return + } + currentSample := math.Log(float64(latency.Microseconds())) + filteredValue := filter.update(sampleTime, currentSample) + micros := math.Exp(filteredValue) + s.estimatedLatencyMicros.Store(uint64(micros)) + // Update the metrics in seconds. + estimateTSOLatencyGauge.WithLabelValues(s.streamID).Set(micros * 1e-6) + } + recvLoop: for { select { @@ -383,14 +411,15 @@ recvLoop: hasReq = false } - durationSeconds := time.Since(currentReq.startTime).Seconds() + latency := time.Since(currentReq.startTime) + latencySeconds := latency.Seconds() if err != nil { // If a request is pending and error occurs, observe the duration it has cost. // Note that it's also possible that the stream is broken due to network without being requested. In this // case, `Recv` may return an error while no request is pending. if hasReq { - requestFailedDurationTSO.Observe(durationSeconds) + requestFailedDurationTSO.Observe(latencySeconds) } if err == io.EOF { finishWithErr = errors.WithStack(errs.ErrClientTSOStreamClosed) @@ -403,9 +432,9 @@ recvLoop: break recvLoop } - latencySeconds := durationSeconds requestDurationTSO.Observe(latencySeconds) tsoBatchSize.Observe(float64(res.count)) + updateEstimatedLatency(currentReq.startTime, latency) if res.count != uint32(currentReq.count) { finishWithErr = errors.WithStack(errTSOLength) @@ -421,6 +450,28 @@ recvLoop: } } +// EstimatedRPCLatency returns an estimation of the duration of each TSO RPC. If the stream has never handled any RPC, +// this function returns 0. +func (s *tsoStream) EstimatedRPCLatency() time.Duration { + failpoint.Inject("tsoStreamSimulateEstimatedRPCLatency", func(val failpoint.Value) { + if s, ok := val.(string); ok { + duration, err := time.ParseDuration(s) + if err != nil { + panic(err) + } + failpoint.Return(duration) + } else { + panic("invalid failpoint value for `tsoStreamSimulateEstimatedRPCLatency`: expected string") + } + }) + latencyUs := s.estimatedLatencyMicros.Load() + // Limit it at least 100us + if latencyUs < 100 { + latencyUs = 100 + } + return time.Microsecond * time.Duration(latencyUs) +} + // GetRecvError returns the error (if any) that has been encountered when receiving response asynchronously. func (s *tsoStream) GetRecvError() error { perr := s.stoppedWithErr.Load() @@ -434,3 +485,48 @@ func (s *tsoStream) GetRecvError() error { func (s *tsoStream) WaitForClosed() { s.wg.Wait() } + +// rcFilter is a simple implementation of a discrete-time low-pass filter. +// Ref: https://en.wikipedia.org/wiki/Low-pass_filter#Simple_infinite_impulse_response_filter +// There are some differences between this implementation and the wikipedia one: +// - Time-interval between each two samples is not necessarily a constant. We allow non-even sample interval by simply +// calculating the alpha (which is calculated by `dt / (rc + dt)`) dynamically for each sample, at the expense of +// losing some mathematical strictness. +// - Support specifying the upperbound of the new sample when updating. This can be an approach to avoid the output +// jumps drastically when the samples come in a low frequency. +type rcFilter struct { + rc float64 + newSampleWeightUpperBound float64 + value float64 + lastSampleTime time.Time + firstSampleArrived bool +} + +// newRCFilter initializes an rcFilter. `cutoff` is the cutoff frequency in Hertz. `newSampleWeightUpperbound` controls +// the upper limit of the weight of each incoming sample (pass 1 for unlimited). +func newRCFilter(cutoff float64, newSampleWeightUpperBound float64) rcFilter { + rc := 1.0 / (2.0 * math.Pi * cutoff) + return rcFilter{ + rc: rc, + newSampleWeightUpperBound: newSampleWeightUpperBound, + } +} + +func (f *rcFilter) update(sampleTime time.Time, newSample float64) float64 { + // Handle the first sample + if !f.firstSampleArrived { + f.firstSampleArrived = true + f.lastSampleTime = sampleTime + f.value = newSample + return newSample + } + + // Delta time. + dt := sampleTime.Sub(f.lastSampleTime).Seconds() + // `alpha` is the weight of the new sample, limited with `newSampleWeightUpperBound`. + alpha := math.Min(dt/(f.rc+dt), f.newSampleWeightUpperBound) + f.value = (1-alpha)*f.value + alpha*newSample + + f.lastSampleTime = sampleTime + return f.value +} diff --git a/client/tso_stream_test.go b/client/tso_stream_test.go index b09c54baf3a0..ab6f2786ff30 100644 --- a/client/tso_stream_test.go +++ b/client/tso_stream_test.go @@ -17,6 +17,7 @@ package pd import ( "context" "io" + "math" "testing" "time" @@ -42,6 +43,14 @@ type resultMsg struct { breakStream bool } +type resultMode int + +const ( + resultModeManual resultMode = iota + resultModeGenerated + resultModeGenerateOnSignal +) + type mockTSOStreamImpl struct { ctx context.Context requestCh chan requestMsg @@ -49,21 +58,21 @@ type mockTSOStreamImpl struct { keyspaceID uint32 errorState error - autoGenerateResult bool + resultMode resultMode // Current progress of generating TSO results resGenPhysical, resGenLogical int64 } -func newMockTSOStreamImpl(ctx context.Context, autoGenerateResult bool) *mockTSOStreamImpl { +func newMockTSOStreamImpl(ctx context.Context, resultMode resultMode) *mockTSOStreamImpl { return &mockTSOStreamImpl{ ctx: ctx, requestCh: make(chan requestMsg, 64), resultCh: make(chan resultMsg, 64), keyspaceID: 0, - autoGenerateResult: autoGenerateResult, - resGenPhysical: 10000, - resGenLogical: 0, + resultMode: resultMode, + resGenPhysical: 10000, + resGenLogical: 0, } } @@ -82,6 +91,17 @@ func (s *mockTSOStreamImpl) Send(clusterID uint64, _keyspaceID, keyspaceGroupID } func (s *mockTSOStreamImpl) Recv() (tsoRequestResult, error) { + var needGenerateResult, needResultSignal bool + switch s.resultMode { + case resultModeManual: + needResultSignal = true + case resultModeGenerated: + needGenerateResult = true + case resultModeGenerateOnSignal: + needResultSignal = true + needGenerateResult = true + } + // This stream have ever receive an error, it returns the error forever. if s.errorState != nil { return tsoRequestResult{}, s.errorState @@ -130,12 +150,12 @@ func (s *mockTSOStreamImpl) Recv() (tsoRequestResult, error) { } s.errorState = res.err return tsoRequestResult{}, s.errorState - } else if s.autoGenerateResult { + } else if !needResultSignal { // Do not allow manually assigning result. panic("trying manually specifying result for mockTSOStreamImpl when it's auto-generating mode") } - } else if s.autoGenerateResult { - res = s.autoGenResult(req.count) + } else if !needResultSignal { + // Mark hasRes as true to skip receiving from resultCh. The actual value of the result will be generated later. hasRes = true } @@ -160,6 +180,10 @@ func (s *mockTSOStreamImpl) Recv() (tsoRequestResult, error) { } } + if needGenerateResult { + res = s.autoGenResult(req.count) + } + // Both res and req should be ready here. if res.err != nil { s.errorState = res.err @@ -168,11 +192,14 @@ func (s *mockTSOStreamImpl) Recv() (tsoRequestResult, error) { } func (s *mockTSOStreamImpl) autoGenResult(count int64) resultMsg { + if count >= (1 << 18) { + panic("requested count too large") + } physical := s.resGenPhysical logical := s.resGenLogical + count if logical >= (1 << 18) { - physical += logical >> 18 - logical &= (1 << 18) - 1 + physical += 1 + logical = count } s.resGenPhysical = physical @@ -190,6 +217,9 @@ func (s *mockTSOStreamImpl) autoGenResult(count int64) resultMsg { } func (s *mockTSOStreamImpl) returnResult(physical int64, logical int64, count uint32) { + if s.resultMode != resultModeManual { + panic("trying to manually specifying tso result on generating mode") + } s.resultCh <- resultMsg{ r: tsoRequestResult{ physical: physical, @@ -201,6 +231,13 @@ func (s *mockTSOStreamImpl) returnResult(physical int64, logical int64, count ui } } +func (s *mockTSOStreamImpl) generateNext() { + if s.resultMode != resultModeGenerateOnSignal { + panic("trying to signal generation when the stream is not generate-on-signal mode") + } + s.resultCh <- resultMsg{} +} + func (s *mockTSOStreamImpl) returnError(err error) { s.resultCh <- resultMsg{ err: err, @@ -233,7 +270,7 @@ type testTSOStreamSuite struct { func (s *testTSOStreamSuite) SetupTest() { s.re = require.New(s.T()) - s.inner = newMockTSOStreamImpl(context.Background(), false) + s.inner = newMockTSOStreamImpl(context.Background(), resultModeManual) s.stream = newTSOStream(context.Background(), mockStreamURL, s.inner) } @@ -454,10 +491,125 @@ func (s *testTSOStreamSuite) TestTSOStreamConcurrentRunning() { } } +func (s *testTSOStreamSuite) TestEstimatedLatency() { + s.inner.returnResult(100, 0, 1) + res := s.getResult(s.mustProcessRequestWithResultCh(1)) + s.re.NoError(res.err) + s.re.Equal(int64(100), res.result.physical) + s.re.Equal(int64(0), res.result.logical) + estimation := s.stream.EstimatedRPCLatency().Seconds() + s.re.Greater(estimation, 0.0) + s.re.InDelta(0.0, estimation, 0.01) + + // For each began request, record its startTime and send it to the result returning goroutine. + reqStartTimeCh := make(chan time.Time, maxPendingRequestsInTSOStream) + // Limit concurrent requests to be less than the capacity of tsoStream.pendingRequests. + tokenCh := make(chan struct{}, maxPendingRequestsInTSOStream-1) + for i := 0; i < 40; i++ { + tokenCh <- struct{}{} + } + // Return a result after 50ms delay for each requests + const delay = time.Millisecond * 50 + // The goroutine to delay and return the result. + go func() { + allocated := int64(1) + for reqStartTime := range reqStartTimeCh { + now := time.Now() + elapsed := now.Sub(reqStartTime) + if elapsed < delay { + time.Sleep(delay - elapsed) + } + s.inner.returnResult(100, allocated, 1) + allocated++ + } + }() + + // Limit the test time within 1s + startTime := time.Now() + resCh := make(chan (<-chan callbackInvocation), 100) + // The sending goroutine + go func() { + for time.Since(startTime) < time.Second { + <-tokenCh + reqStartTimeCh <- time.Now() + r := s.mustProcessRequestWithResultCh(1) + resCh <- r + } + close(reqStartTimeCh) + close(resCh) + }() + // Check the result + index := 0 + for r := range resCh { + // The first is 1 + index++ + res := s.getResult(r) + tokenCh <- struct{}{} + s.re.NoError(res.err) + s.re.Equal(int64(100), res.result.physical) + s.re.Equal(int64(index), res.result.logical) + } + + s.re.Greater(s.stream.EstimatedRPCLatency(), time.Duration(int64(0.9*float64(delay)))) + s.re.Less(s.stream.EstimatedRPCLatency(), time.Duration(math.Floor(1.1*float64(delay)))) +} + +func TestRCFilter(t *testing.T) { + re := require.New(t) + // Test basic calculation with frequency 1 + f := newRCFilter(1, 1) + now := time.Now() + // The first sample initializes the value. + re.Equal(10.0, f.update(now, 10)) + now = now.Add(time.Second) + expectedValue := 10 / (2*math.Pi + 1) + re.InEpsilon(expectedValue, f.update(now, 0), 1e-8) + expectedValue = expectedValue*(1/(2*math.Pi))/(1/(2*math.Pi)+2) + 100*2/(1/(2*math.Pi)+2) + now = now.Add(time.Second * 2) + re.InEpsilon(expectedValue, f.update(now, 100), 1e-8) + + // Test newSampleWeightUpperBound + f = newRCFilter(10, 0.5) + now = time.Now() + re.Equal(0.0, f.update(now, 0)) + now = now.Add(time.Second) + re.InEpsilon(1.0, f.update(now, 2), 1e-8) + now = now.Add(time.Second * 2) + re.InEpsilon(3.0, f.update(now, 5), 1e-8) + + // Test another cutoff frequency and weight upperbound. + f = newRCFilter(1/(2*math.Pi), 0.9) + now = time.Now() + re.Equal(1.0, f.update(now, 1)) + now = now.Add(time.Second) + re.InEpsilon(2.0, f.update(now, 3), 1e-8) + now = now.Add(time.Second * 2) + re.InEpsilon(6.0, f.update(now, 8), 1e-8) + now = now.Add(time.Minute) + re.InEpsilon(15.0, f.update(now, 16), 1e-8) + + // Test with dense samples + f = newRCFilter(1/(2*math.Pi), 0.9) + now = time.Now() + re.Equal(0.0, f.update(now, 0)) + lastOutput := 0.0 + // 10000 even samples in 1 second. + for i := 0; i < 10000; i++ { + now = now.Add(time.Microsecond * 100) + output := f.update(now, 1.0) + re.Greater(output, lastOutput) + re.Less(output, 1.0) + lastOutput = output + } + // Regarding the above samples as being close enough to a continuous function, the output after 1 second + // should be 1 - exp(-RC*t) = 1 - exp(-t). Here RC = 1/(2*pi*cutoff) = 1. + re.InDelta(0.63, lastOutput, 0.02) +} + func BenchmarkTSOStreamSendRecv(b *testing.B) { log.SetLevel(zapcore.FatalLevel) - streamInner := newMockTSOStreamImpl(context.Background(), true) + streamInner := newMockTSOStreamImpl(context.Background(), resultModeGenerated) stream := newTSOStream(context.Background(), mockStreamURL, streamInner) defer func() { streamInner.stop() diff --git a/go.mod b/go.mod index 0d9c5fa0c60a..8a57cfc4994a 100644 --- a/go.mod +++ b/go.mod @@ -23,7 +23,7 @@ require ( github.com/gin-contrib/cors v1.6.0 github.com/gin-contrib/gzip v0.0.1 github.com/gin-contrib/pprof v1.4.0 - github.com/gin-gonic/gin v1.9.1 + github.com/gin-gonic/gin v1.10.0 github.com/gogo/protobuf v1.3.2 github.com/google/btree v1.1.2 github.com/gorilla/mux v1.7.4 @@ -37,7 +37,7 @@ require ( github.com/pingcap/kvproto v0.0.0-20240910154453-b242104f8d31 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-20240830080600-3d0c3db0d55c + github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf github.com/prometheus/client_golang v1.19.0 github.com/prometheus/common v0.51.1 github.com/sasha-s/go-deadlock v0.2.0 @@ -46,7 +46,7 @@ require ( github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.8.4 + github.com/stretchr/testify v1.9.0 github.com/swaggo/http-swagger v1.2.6 github.com/swaggo/swag v1.8.3 github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 @@ -85,11 +85,12 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect - github.com/bytedance/sonic v1.11.2 // indirect + github.com/bytedance/sonic v1.11.6 // indirect + github.com/bytedance/sonic/loader v0.1.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d // indirect - github.com/chenzhuoyu/iasm v0.9.1 // indirect + github.com/cloudwego/base64x v0.1.4 // indirect + github.com/cloudwego/iasm v0.2.0 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.3 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -110,7 +111,7 @@ require ( github.com/go-openapi/swag v0.19.15 // indirect github.com/go-playground/locales v0.14.1 // indirect github.com/go-playground/universal-translator v0.18.1 // indirect - github.com/go-playground/validator/v10 v10.19.0 // indirect + github.com/go-playground/validator/v10 v10.20.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.1.3 // indirect @@ -147,7 +148,7 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/onsi/gomega v1.20.1 // indirect - github.com/pelletier/go-toml/v2 v2.1.1 // indirect + github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect @@ -162,7 +163,7 @@ require ( github.com/shoenig/go-m1cpu v0.1.5 // indirect github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect github.com/sirupsen/logrus v1.9.3 // indirect - github.com/stretchr/objx v0.5.0 // indirect + github.com/stretchr/objx v0.5.2 // indirect github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 // indirect github.com/tidwall/gjson v1.9.3 // indirect github.com/tklauser/go-sysconf v0.3.11 // indirect @@ -191,7 +192,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 - golang.org/x/arch v0.7.0 // indirect + golang.org/x/arch v0.8.0 // indirect golang.org/x/crypto v0.23.0 // indirect golang.org/x/image v0.18.0 // indirect golang.org/x/mod v0.17.0 // indirect @@ -205,7 +206,7 @@ require ( google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index b9977d126c36..943820a10e12 100644 --- a/go.sum +++ b/go.sum @@ -67,10 +67,10 @@ github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpf github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= github.com/brianvoe/gofakeit/v6 v6.26.3 h1:3ljYrjPwsUNAUFdUIr2jVg5EhKdcke/ZLop7uVg1Er8= github.com/brianvoe/gofakeit/v6 v6.26.3/go.mod h1:Xj58BMSnFqcn/fAQeSK+/PLtC5kSb7FJIq4JyGa8vEs= -github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= -github.com/bytedance/sonic v1.11.2 h1:ywfwo0a/3j9HR8wsYGWsIWl2mvRsI950HyoxiBERw5A= -github.com/bytedance/sonic v1.11.2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= +github.com/bytedance/sonic v1.11.6 h1:oUp34TzMlL+OY1OUWxHqsdkgC/Zfc85zGqw9siXjrc0= +github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= +github.com/bytedance/sonic/loader v0.1.1 h1:c+e5Pt1k/cy5wMveRDyk2X4B9hF4g7an8N3zCYjJFNM= +github.com/bytedance/sonic/loader v0.1.1/go.mod h1:ncP89zfokxS5LZrJxl5z0UJcsk4M4yY2JpfqGeCtNLU= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -78,17 +78,14 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ0g/qfRdp61a3Uu/AWrgIq2s0ClJV1g0= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= -github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= -github.com/chenzhuoyu/iasm v0.9.1 h1:tUHQJXo3NhBqw6s33wkGn9SP3bvrWLdlVIJ3hQBL7P0= -github.com/chenzhuoyu/iasm v0.9.1/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/0Y= +github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= +github.com/cloudwego/iasm v0.2.0 h1:1KNIy1I1H9hNNFEEH3DVnI4UujN+1zjpuk6gwHLTssg= +github.com/cloudwego/iasm v0.2.0/go.mod h1:8rXZaNYT2n95jn+zTI1sDr+IgcD2GVs0nlbbQPiEFhY= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa h1:jQCWAUqqlij9Pgj2i/PB79y4KOPYVyFYdROxgaCwdTQ= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq8dk6e9PdstVsDgu9RuyIIJqAaF//0IM= @@ -146,8 +143,8 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= -github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= -github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= +github.com/gin-gonic/gin v1.10.0 h1:nTuyha1TYqgedzytsKYqna+DfLos46nTv2ygFy86HFU= +github.com/gin-gonic/gin v1.10.0/go.mod h1:4PMNQiOhvDRa013RKVbsiNwoyezlm2rm0uX/T7kzp5Y= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -180,8 +177,8 @@ github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJn github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= -github.com/go-playground/validator/v10 v10.19.0 h1:ol+5Fu+cSq9JD7SoSqe04GMI92cbn0+wvQ3bZ8b/AU4= -github.com/go-playground/validator/v10 v10.19.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= +github.com/go-playground/validator/v10 v10.20.0 h1:K9ISHbSaI0lyB2eWMPJo+kOS/FBExVwjEviJTixqxL8= +github.com/go-playground/validator/v10 v10.20.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -375,8 +372,8 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= -github.com/pelletier/go-toml/v2 v2.1.1 h1:LWAJwfNvjQZCFIDKWYQaM62NcYeYViCmWIwmOStowAI= -github.com/pelletier/go-toml/v2 v2.1.1/go.mod h1:tJU2Z3ZkXwnxa4DPO899bsyIoywizdUvyaeZurnPPDc= +github.com/pelletier/go-toml/v2 v2.2.2 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= +github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -400,8 +397,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c h1:+0Tl9izTX2X5cy4wO4cbvjjsg8LqKpHdH5bEMv7pTb4= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf h1:Eh4U5EAP0Hf/h8ApbwXiDwaKzl0wurQrhwS7eaNjyiM= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -468,8 +465,9 @@ github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -480,8 +478,9 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -598,8 +597,8 @@ go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.7.0 h1:pskyeJh/3AmoQ8CPE95vxHLqp1G1GfGNXTmcl9NEKTc= -golang.org/x/arch v0.7.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= +golang.org/x/arch v0.8.0 h1:3wRIsP3pM4yUptoR96otTUOXI367OS0+c9eeRi9doIc= +golang.org/x/arch v0.8.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -773,8 +772,8 @@ google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJai google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/pkg/schedule/schedulers/evict_leader.go b/pkg/schedule/schedulers/evict_leader.go index 28309acddf57..d3c9e1ebff25 100644 --- a/pkg/schedule/schedulers/evict_leader.go +++ b/pkg/schedule/schedulers/evict_leader.go @@ -175,6 +175,12 @@ func (conf *evictLeaderSchedulerConfig) pauseLeaderTransferIfStoreNotExist(id ui return true, nil } +func (conf *evictLeaderSchedulerConfig) resumeLeaderTransferIfExist(id uint64) { + conf.RLock() + defer conf.RUnlock() + conf.cluster.ResumeLeaderTransfer(id) +} + func (conf *evictLeaderSchedulerConfig) update(id uint64, newRanges []core.KeyRange, batch int) error { conf.Lock() defer conf.Unlock() @@ -408,6 +414,7 @@ func (handler *evictLeaderHandler) updateConfig(w http.ResponseWriter, r *http.R batchFloat, ok := input["batch"].(float64) if ok { if batchFloat < 1 || batchFloat > 10 { + handler.config.resumeLeaderTransferIfExist(id) handler.rd.JSON(w, http.StatusBadRequest, "batch is invalid, it should be in [1, 10]") return } @@ -417,6 +424,7 @@ func (handler *evictLeaderHandler) updateConfig(w http.ResponseWriter, r *http.R ranges, ok := (input["ranges"]).([]string) if ok { if !inputHasStoreID { + handler.config.resumeLeaderTransferIfExist(id) handler.rd.JSON(w, http.StatusInternalServerError, errs.ErrSchedulerConfig.FastGenByArgs("id")) return } @@ -426,10 +434,12 @@ func (handler *evictLeaderHandler) updateConfig(w http.ResponseWriter, r *http.R newRanges, err = getKeyRanges(ranges) if err != nil { + handler.config.resumeLeaderTransferIfExist(id) handler.rd.JSON(w, http.StatusInternalServerError, err.Error()) return } + // StoreIDWithRanges is only changed in update function. err = handler.config.update(id, newRanges, batch) if err != nil { handler.rd.JSON(w, http.StatusInternalServerError, err.Error()) diff --git a/pkg/schedule/schedulers/grant_leader.go b/pkg/schedule/schedulers/grant_leader.go index 55b51a14cba7..3fb8225be397 100644 --- a/pkg/schedule/schedulers/grant_leader.go +++ b/pkg/schedule/schedulers/grant_leader.go @@ -271,12 +271,15 @@ func (handler *grantLeaderHandler) updateConfig(w http.ResponseWriter, r *http.R err := handler.config.buildWithArgs(args) if err != nil { + handler.config.Lock() + handler.config.cluster.ResumeLeaderTransfer(id) + handler.config.Unlock() handler.rd.JSON(w, http.StatusBadRequest, err.Error()) return } err = handler.config.persist() if err != nil { - handler.config.removeStore(id) + _, _ = handler.config.removeStore(id) handler.rd.JSON(w, http.StatusInternalServerError, err.Error()) return } diff --git a/pkg/schedule/schedulers/scatter_range.go b/pkg/schedule/schedulers/scatter_range.go index 253675859b71..2e062126feac 100644 --- a/pkg/schedule/schedulers/scatter_range.go +++ b/pkg/schedule/schedulers/scatter_range.go @@ -90,7 +90,7 @@ func (conf *scatterRangeSchedulerConfig) getEndKey() []byte { func (conf *scatterRangeSchedulerConfig) getSchedulerName() string { conf.RLock() defer conf.RUnlock() - return fmt.Sprintf("scatter-range-%s", conf.RangeName) + return fmt.Sprintf("%s-%s", types.ScatterRangeScheduler, conf.RangeName) } type scatterRangeScheduler struct { diff --git a/pkg/schedule/schedulers/scheduler.go b/pkg/schedule/schedulers/scheduler.go index ac0a52fc9779..3f722f7f8047 100644 --- a/pkg/schedule/schedulers/scheduler.go +++ b/pkg/schedule/schedulers/scheduler.go @@ -182,5 +182,12 @@ func FindSchedulerTypeByName(name string) types.CheckerSchedulerType { } } } + // This is for compatibility. Because the string of ScatterRangeScheduler is + // "scatter-range" before. If user adds a ScatterRangeScheduler(which is + // "scatter-range" yet) and then upgrades the cluster(its ScatterRangeScheduler + // is "scatter-range-scheduler"), we need these codes to keep the compatibility. + if len(typ) == 0 && strings.Contains(name, "scatter-range") { + return types.ScatterRangeScheduler + } return typ } diff --git a/pkg/schedule/types/type.go b/pkg/schedule/types/type.go index 09491c5c1c41..7bc278920106 100644 --- a/pkg/schedule/types/type.go +++ b/pkg/schedule/types/type.go @@ -57,8 +57,7 @@ const ( // RandomMergeScheduler is random merge scheduler name. RandomMergeScheduler CheckerSchedulerType = "random-merge-scheduler" // ScatterRangeScheduler is scatter range scheduler name. - // TODO: update to `scatter-range-scheduler` - ScatterRangeScheduler CheckerSchedulerType = "scatter-range" + ScatterRangeScheduler CheckerSchedulerType = "scatter-range-scheduler" // ShuffleHotRegionScheduler is shuffle hot region scheduler name. ShuffleHotRegionScheduler CheckerSchedulerType = "shuffle-hot-region-scheduler" // ShuffleLeaderScheduler is shuffle leader scheduler name. @@ -135,7 +134,8 @@ var ( "grant-hot-region-scheduler": GrantHotRegionScheduler, "balance-hot-region-scheduler": BalanceHotRegionScheduler, "random-merge-scheduler": RandomMergeScheduler, - // TODO: update to `scatter-range-scheduler` + "scatter-range-scheduler": ScatterRangeScheduler, + // TODO: remove `scatter-range` after remove `NewScatterRangeSchedulerCommand` from pd-ctl "scatter-range": ScatterRangeScheduler, "shuffle-hot-region-scheduler": ShuffleHotRegionScheduler, "shuffle-leader-scheduler": ShuffleLeaderScheduler, diff --git a/plugin/scheduler_example/evict_leader.go b/plugin/scheduler_example/evict_leader.go index ff3cd8619733..901dd94c9832 100644 --- a/plugin/scheduler_example/evict_leader.go +++ b/plugin/scheduler_example/evict_leader.go @@ -275,11 +275,18 @@ func (handler *evictLeaderHandler) updateConfig(w http.ResponseWriter, r *http.R err := handler.config.BuildWithArgs(args) if err != nil { + handler.config.mu.Lock() + handler.config.cluster.ResumeLeaderTransfer(id) + handler.config.mu.Unlock() handler.rd.JSON(w, http.StatusBadRequest, err.Error()) return } err = handler.config.Persist() if err != nil { + handler.config.mu.Lock() + delete(handler.config.StoreIDWitRanges, id) + handler.config.cluster.ResumeLeaderTransfer(id) + handler.config.mu.Unlock() handler.rd.JSON(w, http.StatusInternalServerError, err.Error()) return } @@ -301,7 +308,7 @@ func (handler *evictLeaderHandler) deleteConfig(w http.ResponseWriter, r *http.R handler.config.mu.Lock() defer handler.config.mu.Unlock() - _, exists := handler.config.StoreIDWitRanges[id] + ranges, exists := handler.config.StoreIDWitRanges[id] if !exists { handler.rd.JSON(w, http.StatusInternalServerError, errors.New("the config does not exist")) return @@ -309,14 +316,12 @@ func (handler *evictLeaderHandler) deleteConfig(w http.ResponseWriter, r *http.R delete(handler.config.StoreIDWitRanges, id) handler.config.cluster.ResumeLeaderTransfer(id) - handler.config.mu.Unlock() if err := handler.config.Persist(); err != nil { - handler.config.mu.Lock() + handler.config.StoreIDWitRanges[id] = ranges + _ = handler.config.cluster.PauseLeaderTransfer(id) handler.rd.JSON(w, http.StatusInternalServerError, err.Error()) return } - handler.config.mu.Lock() - var resp any if len(handler.config.StoreIDWitRanges) == 0 { resp = noStoreInSchedulerInfo diff --git a/scripts/dashboard-version b/scripts/dashboard-version index 1f6c34ca2fdc..5cb893f415e5 100644 --- a/scripts/dashboard-version +++ b/scripts/dashboard-version @@ -1,3 +1,3 @@ # This file is updated by running scripts/update-dashboard.sh # Don't edit it manually -8.4.0-3d0c3db0 +8.4.0-618b5cde diff --git a/server/api/scheduler.go b/server/api/scheduler.go index c96e4c123dee..3d69674db48a 100644 --- a/server/api/scheduler.go +++ b/server/api/scheduler.go @@ -24,7 +24,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/types" + "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" "github.com/unrolled/render" @@ -143,6 +145,15 @@ func (h *schedulerHandler) CreateScheduler(w http.ResponseWriter, r *http.Reques } collector(strconv.FormatUint(limit, 10)) case types.GrantHotRegionScheduler: + isExist, err := h.isSchedulerExist(types.BalanceHotRegionScheduler) + if err != nil { + h.r.JSON(w, http.StatusInternalServerError, err.Error()) + return + } + if isExist { + h.r.JSON(w, http.StatusBadRequest, "balance-hot-region-scheduler is running, please remove it first") + return + } leaderID, ok := input["store-leader-id"].(string) if !ok { h.r.JSON(w, http.StatusBadRequest, "missing leader id") @@ -155,6 +166,16 @@ func (h *schedulerHandler) CreateScheduler(w http.ResponseWriter, r *http.Reques } collector(leaderID) collector(peerIDs) + case types.BalanceHotRegionScheduler: + isExist, err := h.isSchedulerExist(types.GrantHotRegionScheduler) + if err != nil { + h.r.JSON(w, http.StatusInternalServerError, err.Error()) + return + } + if isExist { + h.r.JSON(w, http.StatusBadRequest, "grant-hot-region-scheduler is running, please remove it first") + return + } } if err := h.AddScheduler(tp, args...); err != nil { @@ -246,6 +267,23 @@ func (h *schedulerHandler) PauseOrResumeScheduler(w http.ResponseWriter, r *http h.r.JSON(w, http.StatusOK, "Pause or resume the scheduler successfully.") } +func (h *schedulerHandler) isSchedulerExist(scheduler types.CheckerSchedulerType) (bool, error) { + rc, err := h.GetRaftCluster() + if err != nil { + return false, err + } + if rc.IsServiceIndependent(constant.SchedulingServiceName) { + handlers := rc.GetSchedulerHandlers() + _, ok := handlers[scheduler.String()] + return ok, nil + } + schedulers := rc.GetSchedulers() + if slice.Contains(schedulers, scheduler.String()) { + return !rc.GetSchedulerConfig().IsSchedulerDisabled(scheduler), nil + } + return false, nil +} + type schedulerConfigHandler struct { svr *server.Server rd *render.Render diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 9a9420988a1d..fa1b14d107e6 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -153,7 +153,9 @@ func TestStoreHeartbeat(t *testing.T) { }, PeerStats: []*pdpb.PeerStat{}, } - cluster.opt.GetScheduleConfig().StoreLimitVersion = "v1" + scheCfg := cluster.opt.GetScheduleConfig().Clone() + scheCfg.StoreLimitVersion = "v1" + cluster.opt.SetScheduleConfig(scheCfg) re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) re.NoError(cluster.HandleStoreHeartbeat(hotReq, hotResp)) diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 65f721776e0a..8b83fdba43dc 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -18,7 +18,7 @@ require ( github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.19.0 github.com/prometheus/client_model v0.6.0 - github.com/stretchr/testify v1.8.4 + github.com/stretchr/testify v1.9.0 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 go.etcd.io/etcd/client/pkg/v3 v3.5.15 @@ -56,13 +56,14 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect - github.com/bytedance/sonic v1.11.2 // indirect + github.com/bytedance/sonic v1.11.6 // indirect + github.com/bytedance/sonic/loader v0.1.1 // indirect github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d // indirect - github.com/chenzhuoyu/iasm v0.9.1 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect + github.com/cloudwego/base64x v0.1.4 // indirect + github.com/cloudwego/iasm v0.2.0 // indirect github.com/coreos/go-semver v0.3.1 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect @@ -74,7 +75,7 @@ require ( github.com/gin-contrib/gzip v0.0.1 // indirect github.com/gin-contrib/pprof v1.4.0 // indirect github.com/gin-contrib/sse v0.1.0 // indirect - github.com/gin-gonic/gin v1.9.1 // indirect + github.com/gin-gonic/gin v1.10.0 // indirect github.com/go-logr/logr v1.3.0 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect @@ -84,7 +85,7 @@ require ( github.com/go-openapi/swag v0.19.15 // indirect github.com/go-playground/locales v0.14.1 // indirect github.com/go-playground/universal-translator v0.18.1 // indirect - github.com/go-playground/validator/v10 v10.19.0 // indirect + github.com/go-playground/validator/v10 v10.20.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/goccy/go-graphviz v0.1.3 // indirect github.com/goccy/go-json v0.10.2 // indirect @@ -125,12 +126,12 @@ require ( github.com/oleiade/reflections v1.0.1 // indirect github.com/opentracing/basictracer-go v1.1.0 github.com/opentracing/opentracing-go v1.2.0 - github.com/pelletier/go-toml/v2 v2.1.1 // indirect + github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -148,7 +149,7 @@ require ( github.com/soheilhy/cmux v0.1.5 // indirect github.com/spf13/cobra v1.8.0 // indirect github.com/spf13/pflag v1.0.5 // indirect - github.com/stretchr/objx v0.5.0 // indirect + github.com/stretchr/objx v0.5.2 // indirect github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 // indirect github.com/swaggo/http-swagger v1.2.6 // indirect github.com/swaggo/swag v1.8.3 // indirect @@ -182,7 +183,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/arch v0.7.0 // indirect + golang.org/x/arch v0.8.0 // indirect golang.org/x/crypto v0.23.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.18.0 // indirect @@ -197,7 +198,7 @@ require ( google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index 223655edd6d2..19cfe26dfc2c 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -65,10 +65,10 @@ github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpf github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= github.com/brianvoe/gofakeit/v6 v6.26.3 h1:3ljYrjPwsUNAUFdUIr2jVg5EhKdcke/ZLop7uVg1Er8= github.com/brianvoe/gofakeit/v6 v6.26.3/go.mod h1:Xj58BMSnFqcn/fAQeSK+/PLtC5kSb7FJIq4JyGa8vEs= -github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= -github.com/bytedance/sonic v1.11.2 h1:ywfwo0a/3j9HR8wsYGWsIWl2mvRsI950HyoxiBERw5A= -github.com/bytedance/sonic v1.11.2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= +github.com/bytedance/sonic v1.11.6 h1:oUp34TzMlL+OY1OUWxHqsdkgC/Zfc85zGqw9siXjrc0= +github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= +github.com/bytedance/sonic/loader v0.1.1 h1:c+e5Pt1k/cy5wMveRDyk2X4B9hF4g7an8N3zCYjJFNM= +github.com/bytedance/sonic/loader v0.1.1/go.mod h1:ncP89zfokxS5LZrJxl5z0UJcsk4M4yY2JpfqGeCtNLU= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -76,19 +76,16 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ0g/qfRdp61a3Uu/AWrgIq2s0ClJV1g0= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= -github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= -github.com/chenzhuoyu/iasm v0.9.1 h1:tUHQJXo3NhBqw6s33wkGn9SP3bvrWLdlVIJ3hQBL7P0= -github.com/chenzhuoyu/iasm v0.9.1/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= +github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/0Y= +github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= +github.com/cloudwego/iasm v0.2.0 h1:1KNIy1I1H9hNNFEEH3DVnI4UujN+1zjpuk6gwHLTssg= +github.com/cloudwego/iasm v0.2.0/go.mod h1:8rXZaNYT2n95jn+zTI1sDr+IgcD2GVs0nlbbQPiEFhY= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa h1:jQCWAUqqlij9Pgj2i/PB79y4KOPYVyFYdROxgaCwdTQ= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq8dk6e9PdstVsDgu9RuyIIJqAaF//0IM= @@ -140,8 +137,8 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= -github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= -github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= +github.com/gin-gonic/gin v1.10.0 h1:nTuyha1TYqgedzytsKYqna+DfLos46nTv2ygFy86HFU= +github.com/gin-gonic/gin v1.10.0/go.mod h1:4PMNQiOhvDRa013RKVbsiNwoyezlm2rm0uX/T7kzp5Y= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -174,8 +171,8 @@ github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJn github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= -github.com/go-playground/validator/v10 v10.19.0 h1:ol+5Fu+cSq9JD7SoSqe04GMI92cbn0+wvQ3bZ8b/AU4= -github.com/go-playground/validator/v10 v10.19.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= +github.com/go-playground/validator/v10 v10.20.0 h1:K9ISHbSaI0lyB2eWMPJo+kOS/FBExVwjEviJTixqxL8= +github.com/go-playground/validator/v10 v10.20.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -372,8 +369,8 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= -github.com/pelletier/go-toml/v2 v2.1.1 h1:LWAJwfNvjQZCFIDKWYQaM62NcYeYViCmWIwmOStowAI= -github.com/pelletier/go-toml/v2 v2.1.1/go.mod h1:tJU2Z3ZkXwnxa4DPO899bsyIoywizdUvyaeZurnPPDc= +github.com/pelletier/go-toml/v2 v2.2.2 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= +github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -397,8 +394,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c h1:+0Tl9izTX2X5cy4wO4cbvjjsg8LqKpHdH5bEMv7pTb4= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf h1:Eh4U5EAP0Hf/h8ApbwXiDwaKzl0wurQrhwS7eaNjyiM= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -462,8 +459,9 @@ github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -474,8 +472,9 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -590,8 +589,8 @@ go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.7.0 h1:pskyeJh/3AmoQ8CPE95vxHLqp1G1GfGNXTmcl9NEKTc= -golang.org/x/arch v0.7.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= +golang.org/x/arch v0.8.0 h1:3wRIsP3pM4yUptoR96otTUOXI367OS0+c9eeRi9doIc= +golang.org/x/arch v0.8.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -762,8 +761,8 @@ google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5u google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index 999781543ff7..2262e483fa2e 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -437,11 +437,15 @@ func (suite *apiTestSuite) checkConfigForward(cluster *tests.TestCluster) { // Test to change config only in scheduling server // Expect to get new config in scheduling server but not old config in api server - opts.GetScheduleConfig().LeaderScheduleLimit = 100 + scheCfg := opts.GetScheduleConfig().Clone() + scheCfg.LeaderScheduleLimit = 100 + opts.SetScheduleConfig(scheCfg) re.Equal(100, int(opts.GetLeaderScheduleLimit())) testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &cfg) re.Equal(100., cfg["schedule"].(map[string]any)["leader-schedule-limit"]) - opts.GetReplicationConfig().MaxReplicas = 5 + repCfg := opts.GetReplicationConfig().Clone() + repCfg.MaxReplicas = 5 + opts.SetReplicationConfig(repCfg) re.Equal(5, int(opts.GetReplicationConfig().MaxReplicas)) testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &cfg) re.Equal(5., cfg["replication"].(map[string]any)["max-replicas"]) diff --git a/tests/server/api/region_test.go b/tests/server/api/region_test.go index 0bec0f43e8be..3a187629830f 100644 --- a/tests/server/api/region_test.go +++ b/tests/server/api/region_test.go @@ -91,7 +91,7 @@ func (suite *regionTestSuite) TearDownTest() { return true }) } - suite.env.RunFuncInTwoModes(cleanFunc) + suite.env.RunTestBasedOnMode(cleanFunc) } func (suite *regionTestSuite) TestSplitRegions() { diff --git a/tests/server/api/rule_test.go b/tests/server/api/rule_test.go index 7d00310d5571..303264d7057e 100644 --- a/tests/server/api/rule_test.go +++ b/tests/server/api/rule_test.go @@ -74,7 +74,7 @@ func (suite *ruleTestSuite) TearDownTest() { err = tu.CheckPostJSON(tests.TestDialClient, urlPrefix+"/pd/api/v1/config/placement-rule", data, tu.StatusOK(re)) re.NoError(err) } - suite.env.RunFuncInTwoModes(cleanFunc) + suite.env.RunTestBasedOnMode(cleanFunc) } func (suite *ruleTestSuite) TestSet() { diff --git a/tests/server/api/scheduler_test.go b/tests/server/api/scheduler_test.go index b989298ee398..b906a72f09e9 100644 --- a/tests/server/api/scheduler_test.go +++ b/tests/server/api/scheduler_test.go @@ -458,8 +458,8 @@ func (suite *scheduleTestSuite) checkAPI(cluster *tests.TestCluster) { }, }, { - name: "scatter-range", - createdName: "scatter-range-test", + name: "scatter-range-scheduler", + createdName: "scatter-range-scheduler-test", args: []arg{{"start_key", ""}, {"end_key", ""}, {"range_name", "test"}}, // Test the scheduler config handler. extraTestFunc: func(name string) { diff --git a/tests/testutil.go b/tests/testutil.go index 51b4a410f8a7..22a5ab40a7ed 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -371,16 +371,6 @@ func (s *SchedulingTestEnvironment) RunTestInAPIMode(test func(*TestCluster)) { test(s.clusters[APIMode]) } -// RunFuncInTwoModes is to run func in two modes. -func (s *SchedulingTestEnvironment) RunFuncInTwoModes(f func(*TestCluster)) { - if c, ok := s.clusters[PDMode]; ok { - f(c) - } - if c, ok := s.clusters[APIMode]; ok { - f(c) - } -} - // Cleanup is to cleanup the environment. func (s *SchedulingTestEnvironment) Cleanup() { for _, cluster := range s.clusters { diff --git a/tools/go.mod b/tools/go.mod index 271115f7f066..14c5c58b7c74 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -16,7 +16,7 @@ require ( github.com/gin-contrib/cors v1.6.0 github.com/gin-contrib/gzip v0.0.1 github.com/gin-contrib/pprof v1.4.0 - github.com/gin-gonic/gin v1.9.1 + github.com/gin-gonic/gin v1.10.0 github.com/go-echarts/go-echarts v1.0.0 github.com/influxdata/tdigest v0.0.1 github.com/mattn/go-shellwords v1.0.12 @@ -28,7 +28,7 @@ require ( github.com/prometheus/common v0.51.1 github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.8.4 + github.com/stretchr/testify v1.9.0 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 go.etcd.io/etcd/client/pkg/v3 v3.5.15 @@ -67,11 +67,12 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect - github.com/bytedance/sonic v1.11.2 // indirect + github.com/bytedance/sonic v1.11.6 // indirect + github.com/bytedance/sonic/loader v0.1.1 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d // indirect - github.com/chenzhuoyu/iasm v0.9.1 // indirect + github.com/cloudwego/base64x v0.1.4 // indirect + github.com/cloudwego/iasm v0.2.0 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/dustin/go-humanize v1.0.1 // indirect @@ -88,7 +89,7 @@ require ( github.com/go-openapi/swag v0.19.15 // indirect github.com/go-playground/locales v0.14.1 // indirect github.com/go-playground/universal-translator v0.18.1 // indirect - github.com/go-playground/validator/v10 v10.19.0 // indirect + github.com/go-playground/validator/v10 v10.20.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.1.3 // indirect @@ -129,12 +130,12 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pelletier/go-toml/v2 v2.1.1 // indirect + github.com/pelletier/go-toml/v2 v2.2.2 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -150,7 +151,7 @@ require ( github.com/sirupsen/logrus v1.9.3 // indirect github.com/smallnest/chanx v1.2.1-0.20240521153536-01121e21ff99 // indirect github.com/soheilhy/cmux v0.1.5 // indirect - github.com/stretchr/objx v0.5.0 // indirect + github.com/stretchr/objx v0.5.2 // indirect github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 // indirect github.com/swaggo/http-swagger v1.2.6 // indirect github.com/swaggo/swag v1.8.3 // indirect @@ -182,7 +183,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/arch v0.7.0 // indirect + golang.org/x/arch v0.8.0 // indirect golang.org/x/crypto v0.23.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.18.0 // indirect @@ -195,7 +196,7 @@ require ( google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/tools/go.sum b/tools/go.sum index cde0d8eeae64..48bb41289e30 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -63,10 +63,10 @@ github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpf github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= github.com/brianvoe/gofakeit/v6 v6.26.3 h1:3ljYrjPwsUNAUFdUIr2jVg5EhKdcke/ZLop7uVg1Er8= github.com/brianvoe/gofakeit/v6 v6.26.3/go.mod h1:Xj58BMSnFqcn/fAQeSK+/PLtC5kSb7FJIq4JyGa8vEs= -github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= -github.com/bytedance/sonic v1.10.0-rc/go.mod h1:ElCzW+ufi8qKqNW0FY314xriJhyJhuoJ3gFZdAHF7NM= -github.com/bytedance/sonic v1.11.2 h1:ywfwo0a/3j9HR8wsYGWsIWl2mvRsI950HyoxiBERw5A= -github.com/bytedance/sonic v1.11.2/go.mod h1:iZcSUejdk5aukTND/Eu/ivjQuEL0Cu9/rf50Hi0u/g4= +github.com/bytedance/sonic v1.11.6 h1:oUp34TzMlL+OY1OUWxHqsdkgC/Zfc85zGqw9siXjrc0= +github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= +github.com/bytedance/sonic/loader v0.1.1 h1:c+e5Pt1k/cy5wMveRDyk2X4B9hF4g7an8N3zCYjJFNM= +github.com/bytedance/sonic/loader v0.1.1/go.mod h1:ncP89zfokxS5LZrJxl5z0UJcsk4M4yY2JpfqGeCtNLU= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -74,13 +74,6 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= -github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d h1:77cEq6EriyTZ0g/qfRdp61a3Uu/AWrgIq2s0ClJV1g0= -github.com/chenzhuoyu/base64x v0.0.0-20230717121745-296ad89f973d/go.mod h1:8EPpVsBuRksnlj1mLy4AWzRNQYxauNi62uWcE3to6eA= -github.com/chenzhuoyu/iasm v0.9.0/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= -github.com/chenzhuoyu/iasm v0.9.1 h1:tUHQJXo3NhBqw6s33wkGn9SP3bvrWLdlVIJ3hQBL7P0= -github.com/chenzhuoyu/iasm v0.9.1/go.mod h1:Xjy2NpN3h7aUqeqM+woSuuvxmIe6+DDsiNLIrkAmYog= github.com/chzyer/logex v1.1.10 h1:Swpa1K6QvQznwJRcfTfQJmTE72DqScAa40E+fbHEXEE= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e h1:fY5BOSpyZCqRo5OhCuC+XN+r/bBCmeuuJtjz+bCNIf8= @@ -88,6 +81,10 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1 h1:q763qf9huN11kDQavWsoZXJNW3xEE4JJyHa5Q25/sd8= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/0Y= +github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= +github.com/cloudwego/iasm v0.2.0 h1:1KNIy1I1H9hNNFEEH3DVnI4UujN+1zjpuk6gwHLTssg= +github.com/cloudwego/iasm v0.2.0/go.mod h1:8rXZaNYT2n95jn+zTI1sDr+IgcD2GVs0nlbbQPiEFhY= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa h1:jQCWAUqqlij9Pgj2i/PB79y4KOPYVyFYdROxgaCwdTQ= github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq8dk6e9PdstVsDgu9RuyIIJqAaF//0IM= @@ -137,8 +134,8 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= -github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= -github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= +github.com/gin-gonic/gin v1.10.0 h1:nTuyha1TYqgedzytsKYqna+DfLos46nTv2ygFy86HFU= +github.com/gin-gonic/gin v1.10.0/go.mod h1:4PMNQiOhvDRa013RKVbsiNwoyezlm2rm0uX/T7kzp5Y= github.com/go-echarts/go-echarts v1.0.0 h1:n181E4iXwj4zrU9VYmdM2m8dyhERt2w9k9YhHqdp6A8= github.com/go-echarts/go-echarts v1.0.0/go.mod h1:qbmyAb/Rl1f2w7wKba1D4LoNq4U164yO4/wedFbcWyo= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -173,8 +170,8 @@ github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJn github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= -github.com/go-playground/validator/v10 v10.19.0 h1:ol+5Fu+cSq9JD7SoSqe04GMI92cbn0+wvQ3bZ8b/AU4= -github.com/go-playground/validator/v10 v10.19.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= +github.com/go-playground/validator/v10 v10.20.0 h1:K9ISHbSaI0lyB2eWMPJo+kOS/FBExVwjEviJTixqxL8= +github.com/go-playground/validator/v10 v10.20.0/go.mod h1:dbuPbCMFw/DrkbEynArYaCwl3amGuJotoKCe95atGMM= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -369,8 +366,8 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= -github.com/pelletier/go-toml/v2 v2.1.1 h1:LWAJwfNvjQZCFIDKWYQaM62NcYeYViCmWIwmOStowAI= -github.com/pelletier/go-toml/v2 v2.1.1/go.mod h1:tJU2Z3ZkXwnxa4DPO899bsyIoywizdUvyaeZurnPPDc= +github.com/pelletier/go-toml/v2 v2.2.2 h1:aYUidT7k73Pcl9nb2gScu7NSrKCSHIDE89b3+6Wq+LM= +github.com/pelletier/go-toml/v2 v2.2.2/go.mod h1:1t835xjRzz80PqgE6HHgN2JOsmgYu/h4qDAS4n929Rs= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -394,8 +391,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c h1:+0Tl9izTX2X5cy4wO4cbvjjsg8LqKpHdH5bEMv7pTb4= -github.com/pingcap/tidb-dashboard v0.0.0-20240830080600-3d0c3db0d55c/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf h1:Eh4U5EAP0Hf/h8ApbwXiDwaKzl0wurQrhwS7eaNjyiM= +github.com/pingcap/tidb-dashboard v0.0.0-20240924035706-618b5cded5bf/go.mod h1:AT9vfeojwr/GGCHTURXtA8yZBE9AW8LdIo02/eYdfHU= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -461,8 +458,9 @@ github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= -github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -474,8 +472,9 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -592,8 +591,8 @@ go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= -golang.org/x/arch v0.7.0 h1:pskyeJh/3AmoQ8CPE95vxHLqp1G1GfGNXTmcl9NEKTc= -golang.org/x/arch v0.7.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= +golang.org/x/arch v0.8.0 h1:3wRIsP3pM4yUptoR96otTUOXI367OS0+c9eeRi9doIc= +golang.org/x/arch v0.8.0/go.mod h1:FEVrYAQjsQXMVJ1nsMoVVXPZg6p2JE2mx8psSWTDQys= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -766,8 +765,8 @@ google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5u google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/tools/pd-ctl/pdctl/command/config_command.go b/tools/pd-ctl/pdctl/command/config_command.go index 4efaae3fa130..a8e5af73638f 100644 --- a/tools/pd-ctl/pdctl/command/config_command.go +++ b/tools/pd-ctl/pdctl/command/config_command.go @@ -17,6 +17,7 @@ package command import ( "bytes" "encoding/json" + "fmt" "net/http" "net/url" "os" @@ -33,19 +34,20 @@ import ( ) const ( - configPrefix = "pd/api/v1/config" - schedulePrefix = "pd/api/v1/config/schedule" - replicatePrefix = "pd/api/v1/config/replicate" - labelPropertyPrefix = "pd/api/v1/config/label-property" - clusterVersionPrefix = "pd/api/v1/config/cluster-version" - rulesPrefix = "pd/api/v1/config/rules" - rulesBatchPrefix = "pd/api/v1/config/rules/batch" - rulePrefix = "pd/api/v1/config/rule" - ruleGroupPrefix = "pd/api/v1/config/rule_group" - ruleGroupsPrefix = "pd/api/v1/config/rule_groups" - replicationModePrefix = "pd/api/v1/config/replication-mode" - ruleBundlePrefix = "pd/api/v1/config/placement-rule" - pdServerPrefix = "pd/api/v1/config/pd-server" + configPrefix = "pd/api/v1/config" + schedulePrefix = "pd/api/v1/config/schedule" + replicatePrefix = "pd/api/v1/config/replicate" + labelPropertyPrefix = "pd/api/v1/config/label-property" + clusterVersionPrefix = "pd/api/v1/config/cluster-version" + rulesPrefix = "pd/api/v1/config/rules" + rulesBatchPrefix = "pd/api/v1/config/rules/batch" + rulePrefix = "pd/api/v1/config/rule" + ruleGroupPrefix = "pd/api/v1/config/rule_group" + ruleGroupsPrefix = "pd/api/v1/config/rule_groups" + replicationModePrefix = "pd/api/v1/config/replication-mode" + ruleBundlePrefix = "pd/api/v1/config/placement-rule" + pdServerPrefix = "pd/api/v1/config/pd-server" + serviceMiddlewareConfigPrefix = "pd/api/v1/service-middleware/config" // flagFromAPIServer has no influence for pd mode, but it is useful for us to debug in api mode. flagFromAPIServer = "from_api_server" ) @@ -77,6 +79,7 @@ func NewShowConfigCommand() *cobra.Command { sc.AddCommand(NewShowClusterVersionCommand()) sc.AddCommand(newShowReplicationModeCommand()) sc.AddCommand(NewShowServerConfigCommand()) + sc.AddCommand(NewShowServiceMiddlewareConfigCommand()) sc.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") return sc } @@ -151,6 +154,16 @@ func NewShowServerConfigCommand() *cobra.Command { } } +// NewShowReplicationConfigCommand return a show all subcommand of show subcommand +func NewShowServiceMiddlewareConfigCommand() *cobra.Command { + sc := &cobra.Command{ + Use: "service-middleware", + Short: "show service middleware config of PD", + Run: showServiceMiddlewareConfigCommandFunc, + } + return sc +} + // NewSetConfigCommand return a set subcommand of configCmd func NewSetConfigCommand() *cobra.Command { sc := &cobra.Command{ @@ -161,6 +174,7 @@ func NewSetConfigCommand() *cobra.Command { sc.AddCommand(NewSetLabelPropertyCommand()) sc.AddCommand(NewSetClusterVersionCommand()) sc.AddCommand(newSetReplicationModeCommand()) + sc.AddCommand(newSetServiceMiddlewareCommand()) return sc } @@ -192,6 +206,14 @@ func newSetReplicationModeCommand() *cobra.Command { } } +func newSetServiceMiddlewareCommand() *cobra.Command { + return &cobra.Command{ + Use: "service-middleware [ |