From 1e76110a1241c954f9d4085ae57c4a853a7f0986 Mon Sep 17 00:00:00 2001 From: Artem Danilov Date: Mon, 9 Dec 2024 20:23:39 -0800 Subject: [PATCH 01/33] client: introduce circuit breaker for region calls (#8856) ref tikv/pd#8678 Signed-off-by: artem_danilov Co-authored-by: artem_danilov Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/circuitbreaker/circuit_breaker.go | 302 ++++++++++++++++++ client/circuitbreaker/circuit_breaker_test.go | 270 ++++++++++++++++ client/client.go | 23 +- client/errs/errno.go | 1 + client/inner_client.go | 23 +- client/metrics/metrics.go | 12 + client/opt/option.go | 28 +- 7 files changed, 647 insertions(+), 12 deletions(-) create mode 100644 client/circuitbreaker/circuit_breaker.go create mode 100644 client/circuitbreaker/circuit_breaker_test.go diff --git a/client/circuitbreaker/circuit_breaker.go b/client/circuitbreaker/circuit_breaker.go new file mode 100644 index 00000000000..b5a4c53ebb5 --- /dev/null +++ b/client/circuitbreaker/circuit_breaker.go @@ -0,0 +1,302 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package circuitbreaker + +import ( + "fmt" + "strings" + "sync" + "time" + + "github.com/tikv/pd/client/errs" + + "github.com/prometheus/client_golang/prometheus" + m "github.com/tikv/pd/client/metrics" + "go.uber.org/zap" + + "github.com/pingcap/log" +) + +// Overloading is a type describing service return value +type Overloading bool + +const ( + // No means the service is not overloaded + No = false + // Yes means the service is overloaded + Yes = true +) + +// Settings describes configuration for Circuit Breaker +type Settings struct { + // Defines the error rate threshold to trip the circuit breaker. + ErrorRateThresholdPct uint32 + // Defines the average qps over the `error_rate_window` that must be met before evaluating the error rate threshold. + MinQPSForOpen uint32 + // Defines how long to track errors before evaluating error_rate_threshold. + ErrorRateWindow time.Duration + // Defines how long to wait after circuit breaker is open before go to half-open state to send a probe request. + CoolDownInterval time.Duration + // Defines how many subsequent requests to test after cooldown period before fully close the circuit. + HalfOpenSuccessCount uint32 +} + +// AlwaysClosedSettings is a configuration that never trips the circuit breaker. +var AlwaysClosedSettings = Settings{ + ErrorRateThresholdPct: 0, // never trips + ErrorRateWindow: 10 * time.Second, // effectively results in testing for new settings every 10 seconds + MinQPSForOpen: 10, + CoolDownInterval: 10 * time.Second, + HalfOpenSuccessCount: 1, +} + +// CircuitBreaker is a state machine to prevent sending requests that are likely to fail. +type CircuitBreaker[T any] struct { + config *Settings + name string + + mutex sync.Mutex + state *State[T] + + successCounter prometheus.Counter + errorCounter prometheus.Counter + overloadCounter prometheus.Counter + fastFailCounter prometheus.Counter +} + +// StateType is a type that represents a state of CircuitBreaker. +type StateType int + +// States of CircuitBreaker. +const ( + StateClosed StateType = iota + StateOpen + StateHalfOpen +) + +// String implements stringer interface. +func (s StateType) String() string { + switch s { + case StateClosed: + return "closed" + case StateOpen: + return "open" + case StateHalfOpen: + return "half-open" + default: + return fmt.Sprintf("unknown state: %d", s) + } +} + +var replacer = strings.NewReplacer(" ", "_", "-", "_") + +// NewCircuitBreaker returns a new CircuitBreaker configured with the given Settings. +func NewCircuitBreaker[T any](name string, st Settings) *CircuitBreaker[T] { + cb := new(CircuitBreaker[T]) + cb.name = name + cb.config = &st + cb.state = cb.newState(time.Now(), StateClosed) + + metricName := replacer.Replace(name) + cb.successCounter = m.CircuitBreakerCounters.WithLabelValues(metricName, "success") + cb.errorCounter = m.CircuitBreakerCounters.WithLabelValues(metricName, "error") + cb.overloadCounter = m.CircuitBreakerCounters.WithLabelValues(metricName, "overload") + cb.fastFailCounter = m.CircuitBreakerCounters.WithLabelValues(metricName, "fast_fail") + return cb +} + +// ChangeSettings changes the CircuitBreaker settings. +// The changes will be reflected only in the next evaluation window. +func (cb *CircuitBreaker[T]) ChangeSettings(apply func(config *Settings)) { + cb.mutex.Lock() + defer cb.mutex.Unlock() + + apply(cb.config) +} + +// Execute calls the given function if the CircuitBreaker is closed and returns the result of execution. +// Execute returns an error instantly if the CircuitBreaker is open. +// https://github.com/tikv/rfcs/blob/master/text/0115-circuit-breaker.md +func (cb *CircuitBreaker[T]) Execute(call func() (T, Overloading, error)) (T, error) { + state, err := cb.onRequest() + if err != nil { + cb.fastFailCounter.Inc() + var defaultValue T + return defaultValue, err + } + + defer func() { + e := recover() + if e != nil { + cb.emitMetric(Yes, err) + cb.onResult(state, Yes) + panic(e) + } + }() + + result, overloaded, err := call() + cb.emitMetric(overloaded, err) + cb.onResult(state, overloaded) + return result, err +} + +func (cb *CircuitBreaker[T]) onRequest() (*State[T], error) { + cb.mutex.Lock() + defer cb.mutex.Unlock() + + state, err := cb.state.onRequest(cb) + cb.state = state + return state, err +} + +func (cb *CircuitBreaker[T]) onResult(state *State[T], overloaded Overloading) { + cb.mutex.Lock() + defer cb.mutex.Unlock() + + // even if the circuit breaker already moved to a new state while the request was in progress, + // it is still ok to update the old state, but it is not relevant anymore + state.onResult(overloaded) +} + +func (cb *CircuitBreaker[T]) emitMetric(overloaded Overloading, err error) { + switch overloaded { + case No: + cb.successCounter.Inc() + case Yes: + cb.overloadCounter.Inc() + default: + panic("unknown state") + } + if err != nil { + cb.errorCounter.Inc() + } +} + +// State represents the state of CircuitBreaker. +type State[T any] struct { + stateType StateType + cb *CircuitBreaker[T] + end time.Time + + pendingCount uint32 + successCount uint32 + failureCount uint32 +} + +// newState creates a new State with the given configuration and reset all success/failure counters. +func (cb *CircuitBreaker[T]) newState(now time.Time, stateType StateType) *State[T] { + var end time.Time + var pendingCount uint32 + switch stateType { + case StateClosed: + end = now.Add(cb.config.ErrorRateWindow) + case StateOpen: + end = now.Add(cb.config.CoolDownInterval) + case StateHalfOpen: + // we transition to HalfOpen state on the first request after the cooldown period, + // so we start with 1 pending request + pendingCount = 1 + default: + panic("unknown state") + } + return &State[T]{ + cb: cb, + stateType: stateType, + pendingCount: pendingCount, + end: end, + } +} + +// onRequest transitions the state to the next state based on the current state and the previous requests results +// The implementation represents a state machine for CircuitBreaker +// All state transitions happens at the request evaluation time only +// Circuit breaker start with a closed state, allows all requests to pass through and always lasts for a fixed duration of `Settings.ErrorRateWindow`. +// If `Settings.ErrorRateThresholdPct` is breached at the end of the window, then it moves to Open state, otherwise it moves to a new Closed state with a new window. +// Open state fails all request, it has a fixed duration of `Settings.CoolDownInterval` and always moves to HalfOpen state at the end of the interval. +// HalfOpen state does not have a fixed duration and lasts till `Settings.HalfOpenSuccessCount` are evaluated. +// If any of `Settings.HalfOpenSuccessCount` fails then it moves back to Open state, otherwise it moves to Closed state. +func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { + var now = time.Now() + switch s.stateType { + case StateClosed: + if now.After(s.end) { + // ErrorRateWindow is over, let's evaluate the error rate + if s.cb.config.ErrorRateThresholdPct > 0 { // otherwise circuit breaker is disabled + total := s.failureCount + s.successCount + if total > 0 { + observedErrorRatePct := s.failureCount * 100 / total + if total >= uint32(s.cb.config.ErrorRateWindow.Seconds())*s.cb.config.MinQPSForOpen && observedErrorRatePct >= s.cb.config.ErrorRateThresholdPct { + // the error threshold is breached, let's move to open state and start failing all requests + log.Error("Circuit breaker tripped. Starting to fail all requests", + zap.String("name", cb.name), + zap.Uint32("observedErrorRatePct", observedErrorRatePct), + zap.String("config", fmt.Sprintf("%+v", cb.config))) + return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen + } + } + } + // the error threshold is not breached or there were not enough requests to evaluate it, + // continue in the closed state and allow all requests + return cb.newState(now, StateClosed), nil + } + // continue in closed state till ErrorRateWindow is over + return s, nil + case StateOpen: + if now.After(s.end) { + // CoolDownInterval is over, it is time to transition to half-open state + log.Info("Circuit breaker cooldown period is over. Transitioning to half-open state to test the service", + zap.String("name", cb.name), + zap.String("config", fmt.Sprintf("%+v", cb.config))) + return cb.newState(now, StateHalfOpen), nil + } else { + // continue in the open state till CoolDownInterval is over + return s, errs.ErrCircuitBreakerOpen + } + case StateHalfOpen: + // do we need some expire time here in case of one of pending requests is stuck forever? + if s.failureCount > 0 { + // there were some failures during half-open state, let's go back to open state to wait a bit longer + log.Error("Circuit breaker goes from half-open to open again as errors persist and continue to fail all requests", + zap.String("name", cb.name), + zap.String("config", fmt.Sprintf("%+v", cb.config))) + return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen + } else if s.successCount == s.cb.config.HalfOpenSuccessCount { + // all probe requests are succeeded, we can move to closed state and allow all requests + log.Info("Circuit breaker is closed. Start allowing all requests", + zap.String("name", cb.name), + zap.String("config", fmt.Sprintf("%+v", cb.config))) + return cb.newState(now, StateClosed), nil + } else if s.pendingCount < s.cb.config.HalfOpenSuccessCount { + // allow more probe requests and continue in half-open state + s.pendingCount++ + return s, nil + } else { + // continue in half-open state till all probe requests are done and fail all other requests for now + return s, errs.ErrCircuitBreakerOpen + } + default: + panic("unknown state") + } +} + +func (s *State[T]) onResult(overloaded Overloading) { + switch overloaded { + case No: + s.successCount++ + case Yes: + s.failureCount++ + default: + panic("unknown state") + } +} diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/circuitbreaker/circuit_breaker_test.go new file mode 100644 index 00000000000..ca77b7f9f99 --- /dev/null +++ b/client/circuitbreaker/circuit_breaker_test.go @@ -0,0 +1,270 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package circuitbreaker + +import ( + "errors" + "testing" + "time" + + "github.com/tikv/pd/client/errs" + + "github.com/stretchr/testify/require" +) + +// advance emulate the state machine clock moves forward by the given duration +func (cb *CircuitBreaker[T]) advance(duration time.Duration) { + cb.state.end = cb.state.end.Add(-duration - 1) +} + +var settings = Settings{ + ErrorRateThresholdPct: 50, + MinQPSForOpen: 10, + ErrorRateWindow: 30 * time.Second, + CoolDownInterval: 10 * time.Second, + HalfOpenSuccessCount: 2, +} + +var minCountToOpen = int(settings.MinQPSForOpen * uint32(settings.ErrorRateWindow.Seconds())) + +func TestCircuitBreaker_Execute_Wrapper_Return_Values(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + originalError := errors.New("circuit breaker is open") + + result, err := cb.Execute(func() (int, Overloading, error) { + return 42, No, originalError + }) + re.Equal(err, originalError) + re.Equal(42, result) + + // same by interpret the result as overloading error + result, err = cb.Execute(func() (int, Overloading, error) { + return 42, Yes, originalError + }) + re.Equal(err, originalError) + re.Equal(42, result) +} + +func TestCircuitBreaker_OpenState(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + driveQPS(cb, minCountToOpen, Yes, re) + re.Equal(StateClosed, cb.state.stateType) + assertSucceeds(cb, re) // no error till ErrorRateWindow is finished + cb.advance(settings.ErrorRateWindow) + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) +} + +func TestCircuitBreaker_CloseState_Not_Enough_QPS(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + driveQPS(cb, minCountToOpen/2, Yes, re) + cb.advance(settings.ErrorRateWindow) + assertSucceeds(cb, re) + re.Equal(StateClosed, cb.state.stateType) +} + +func TestCircuitBreaker_CloseState_Not_Enough_Error_Rate(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + driveQPS(cb, minCountToOpen/4, Yes, re) + driveQPS(cb, minCountToOpen, No, re) + cb.advance(settings.ErrorRateWindow) + assertSucceeds(cb, re) + re.Equal(StateClosed, cb.state.stateType) +} + +func TestCircuitBreaker_Half_Open_To_Closed(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + driveQPS(cb, minCountToOpen, Yes, re) + cb.advance(settings.ErrorRateWindow) + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) + cb.advance(settings.CoolDownInterval) + assertSucceeds(cb, re) + re.Equal(StateHalfOpen, cb.state.stateType) + assertSucceeds(cb, re) + re.Equal(StateHalfOpen, cb.state.stateType) + // state always transferred on the incoming request + assertSucceeds(cb, re) + re.Equal(StateClosed, cb.state.stateType) +} + +func TestCircuitBreaker_Half_Open_To_Open(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + driveQPS(cb, minCountToOpen, Yes, re) + cb.advance(settings.ErrorRateWindow) + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) + cb.advance(settings.CoolDownInterval) + assertSucceeds(cb, re) + re.Equal(StateHalfOpen, cb.state.stateType) + _, err := cb.Execute(func() (int, Overloading, error) { + return 42, Yes, nil // this trip circuit breaker again + }) + re.NoError(err) + re.Equal(StateHalfOpen, cb.state.stateType) + // state always transferred on the incoming request + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) +} + +// in half open state, circuit breaker will allow only HalfOpenSuccessCount pending and should fast fail all other request till HalfOpenSuccessCount requests is completed +// this test moves circuit breaker to the half open state and verifies that requests above HalfOpenSuccessCount are failing +func TestCircuitBreaker_Half_Open_Fail_Over_Pending_Count(t *testing.T) { + re := require.New(t) + cb := newCircuitBreakerMovedToHalfOpenState(re) + + // the next request will move circuit breaker into the half open state + var started []chan bool + var waited []chan bool + var ended []chan bool + for range settings.HalfOpenSuccessCount { + start := make(chan bool) + wait := make(chan bool) + end := make(chan bool) + started = append(started, start) + waited = append(waited, wait) + ended = append(ended, end) + go func() { + defer func() { + end <- true + }() + _, err := cb.Execute(func() (int, Overloading, error) { + start <- true + <-wait + return 42, No, nil + }) + re.NoError(err) + }() + } + // make sure all requests are started + for i := range started { + <-started[i] + } + // validate that requests beyond HalfOpenSuccessCount are failing + assertFastFail(cb, re) + re.Equal(StateHalfOpen, cb.state.stateType) + // unblock pending requests and wait till they are completed + for i := range ended { + waited[i] <- true + <-ended[i] + } + // validate that circuit breaker moves to closed state + assertSucceeds(cb, re) + re.Equal(StateClosed, cb.state.stateType) + // make sure that after moving to open state all counters are reset + re.Equal(uint32(1), cb.state.successCount) +} + +func TestCircuitBreaker_Count_Only_Requests_In_Same_Window(t *testing.T) { + re := require.New(t) + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + + start := make(chan bool) + wait := make(chan bool) + end := make(chan bool) + go func() { + defer func() { + end <- true + }() + _, err := cb.Execute(func() (int, Overloading, error) { + start <- true + <-wait + return 42, No, nil + }) + re.NoError(err) + }() + <-start // make sure the request is started + // assert running request is not counted + re.Equal(uint32(0), cb.state.successCount) + + // advance request to the next window + cb.advance(settings.ErrorRateWindow) + assertSucceeds(cb, re) + re.Equal(uint32(1), cb.state.successCount) + + // complete the request from the previous window + wait <- true // resume + <-end // wait for the request to complete + // assert request from last window is not counted + re.Equal(uint32(1), cb.state.successCount) +} + +func TestCircuitBreaker_ChangeSettings(t *testing.T) { + re := require.New(t) + + cb := NewCircuitBreaker[int]("test_cb", AlwaysClosedSettings) + driveQPS(cb, int(AlwaysClosedSettings.MinQPSForOpen*uint32(AlwaysClosedSettings.ErrorRateWindow.Seconds())), Yes, re) + cb.advance(AlwaysClosedSettings.ErrorRateWindow) + assertSucceeds(cb, re) + re.Equal(StateClosed, cb.state.stateType) + + cb.ChangeSettings(func(config *Settings) { + config.ErrorRateThresholdPct = settings.ErrorRateThresholdPct + }) + re.Equal(settings.ErrorRateThresholdPct, cb.config.ErrorRateThresholdPct) + + driveQPS(cb, minCountToOpen, Yes, re) + cb.advance(settings.ErrorRateWindow) + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) +} + +func newCircuitBreakerMovedToHalfOpenState(re *require.Assertions) *CircuitBreaker[int] { + cb := NewCircuitBreaker[int]("test_cb", settings) + re.Equal(StateClosed, cb.state.stateType) + driveQPS(cb, minCountToOpen, Yes, re) + cb.advance(settings.ErrorRateWindow) + assertFastFail(cb, re) + re.Equal(StateOpen, cb.state.stateType) + cb.advance(settings.CoolDownInterval) + return cb +} + +func driveQPS(cb *CircuitBreaker[int], count int, overload Overloading, re *require.Assertions) { + for range count { + _, err := cb.Execute(func() (int, Overloading, error) { + return 42, overload, nil + }) + re.NoError(err) + } +} + +func assertFastFail(cb *CircuitBreaker[int], re *require.Assertions) { + var executed = false + _, err := cb.Execute(func() (int, Overloading, error) { + executed = true + return 42, No, nil + }) + re.Equal(err, errs.ErrCircuitBreakerOpen) + re.False(executed) +} + +func assertSucceeds(cb *CircuitBreaker[int], re *require.Assertions) { + result, err := cb.Execute(func() (int, Overloading, error) { + return 42, No, nil + }) + re.NoError(err) + re.Equal(42, result) +} diff --git a/client/client.go b/client/client.go index 49ce73bf9fb..c271f10591d 100644 --- a/client/client.go +++ b/client/client.go @@ -22,6 +22,8 @@ import ( "sync" "time" + cb "github.com/tikv/pd/client/circuitbreaker" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -456,6 +458,12 @@ func (c *client) UpdateOption(option opt.DynamicOption, value any) error { return errors.New("[pd] invalid value type for TSOClientRPCConcurrency option, it should be int") } c.inner.option.SetTSOClientRPCConcurrency(value) + case opt.RegionMetadataCircuitBreakerSettings: + applySettingsChange, ok := value.(func(config *cb.Settings)) + if !ok { + return errors.New("[pd] invalid value type for RegionMetadataCircuitBreakerSettings option, it should be pd.Settings") + } + c.inner.regionMetaCircuitBreaker.ChangeSettings(applySettingsChange) default: return errors.New("[pd] unsupported client option") } @@ -650,7 +658,10 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) + resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { + region, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) + return region, isOverloaded(err), err + }) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { @@ -690,7 +701,10 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetR if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetPrevRegion(cctx, req) + resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { + resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetPrevRegion(cctx, req) + return resp, isOverloaded(err), err + }) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { @@ -730,7 +744,10 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegionByID(cctx, req) + resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { + resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegionByID(cctx, req) + return resp, isOverloaded(err), err + }) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { diff --git a/client/errs/errno.go b/client/errs/errno.go index df8b677525a..25665f01017 100644 --- a/client/errs/errno.go +++ b/client/errs/errno.go @@ -70,6 +70,7 @@ var ( ErrClientGetServingEndpoint = errors.Normalize("get serving endpoint failed", errors.RFCCodeText("PD:client:ErrClientGetServingEndpoint")) ErrClientFindGroupByKeyspaceID = errors.Normalize("can't find keyspace group by keyspace id", errors.RFCCodeText("PD:client:ErrClientFindGroupByKeyspaceID")) ErrClientWatchGCSafePointV2Stream = errors.Normalize("watch gc safe point v2 stream failed", errors.RFCCodeText("PD:client:ErrClientWatchGCSafePointV2Stream")) + ErrCircuitBreakerOpen = errors.Normalize("circuit breaker is open", errors.RFCCodeText("PD:client:ErrCircuitBreakerOpen")) ) // grpcutil errors diff --git a/client/inner_client.go b/client/inner_client.go index 7be35e9a3b9..ae15c763854 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -6,9 +6,12 @@ import ( "sync" "time" + "google.golang.org/grpc/codes" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" @@ -16,6 +19,7 @@ import ( sd "github.com/tikv/pd/client/servicediscovery" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/status" ) const ( @@ -24,10 +28,11 @@ const ( ) type innerClient struct { - keyspaceID uint32 - svrUrls []string - pdSvcDiscovery sd.ServiceDiscovery - tokenDispatcher *tokenDispatcher + keyspaceID uint32 + svrUrls []string + pdSvcDiscovery sd.ServiceDiscovery + tokenDispatcher *tokenDispatcher + regionMetaCircuitBreaker *cb.CircuitBreaker[*pdpb.GetRegionResponse] // For service mode switching. serviceModeKeeper @@ -53,6 +58,7 @@ func (c *innerClient) init(updateKeyspaceIDCb sd.UpdateKeyspaceIDFunc) error { } return err } + c.regionMetaCircuitBreaker = cb.NewCircuitBreaker[*pdpb.GetRegionResponse]("region_meta", c.option.RegionMetaCircuitBreakerSettings) return nil } @@ -245,3 +251,12 @@ func (c *innerClient) dispatchTSORequestWithRetry(ctx context.Context) tso.TSFut } return req } + +func isOverloaded(err error) cb.Overloading { + switch status.Code(errors.Cause(err)) { + case codes.DeadlineExceeded, codes.Unavailable, codes.ResourceExhausted: + return cb.Yes + default: + return cb.No + } +} diff --git a/client/metrics/metrics.go b/client/metrics/metrics.go index da36217eb34..3a3199c74a6 100644 --- a/client/metrics/metrics.go +++ b/client/metrics/metrics.go @@ -56,6 +56,8 @@ var ( OngoingRequestCountGauge *prometheus.GaugeVec // EstimateTSOLatencyGauge is the gauge to indicate the estimated latency of TSO requests. EstimateTSOLatencyGauge *prometheus.GaugeVec + // CircuitBreakerCounters is a vector for different circuit breaker counters + CircuitBreakerCounters *prometheus.CounterVec ) func initMetrics(constLabels prometheus.Labels) { @@ -144,6 +146,15 @@ func initMetrics(constLabels prometheus.Labels) { Help: "Estimated latency of an RTT of getting TSO", ConstLabels: constLabels, }, []string{"stream"}) + + CircuitBreakerCounters = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "pd_client", + Subsystem: "request", + Name: "circuit_breaker_count", + Help: "Circuit Breaker counters", + ConstLabels: constLabels, + }, []string{"name", "success"}) } // CmdDurationXXX and CmdFailedDurationXXX are the durations of the client commands. @@ -259,4 +270,5 @@ func registerMetrics() { prometheus.MustRegister(TSOBatchSendLatency) prometheus.MustRegister(RequestForwarded) prometheus.MustRegister(EstimateTSOLatencyGauge) + prometheus.MustRegister(CircuitBreakerCounters) } diff --git a/client/opt/option.go b/client/opt/option.go index faeb232195f..9a80a895cc0 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -18,6 +18,8 @@ import ( "sync/atomic" "time" + cb "github.com/tikv/pd/client/circuitbreaker" + "github.com/pingcap/errors" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/pd/client/pkg/retry" @@ -47,6 +49,8 @@ const ( EnableFollowerHandle // TSOClientRPCConcurrency controls the amount of ongoing TSO RPC requests at the same time in a single TSO client. TSOClientRPCConcurrency + // RegionMetadataCircuitBreakerSettings controls settings for circuit breaker for region metadata requests. + RegionMetadataCircuitBreakerSettings dynamicOptionCount ) @@ -67,16 +71,18 @@ type Option struct { // Dynamic options. dynamicOptions [dynamicOptionCount]atomic.Value - EnableTSOFollowerProxyCh chan struct{} + EnableTSOFollowerProxyCh chan struct{} + RegionMetaCircuitBreakerSettings cb.Settings } // NewOption creates a new PD client option with the default values set. func NewOption() *Option { co := &Option{ - Timeout: defaultPDTimeout, - MaxRetryTimes: maxInitClusterRetries, - EnableTSOFollowerProxyCh: make(chan struct{}, 1), - InitMetrics: true, + Timeout: defaultPDTimeout, + MaxRetryTimes: maxInitClusterRetries, + EnableTSOFollowerProxyCh: make(chan struct{}, 1), + InitMetrics: true, + RegionMetaCircuitBreakerSettings: cb.AlwaysClosedSettings, } co.dynamicOptions[MaxTSOBatchWaitInterval].Store(defaultMaxTSOBatchWaitInterval) @@ -147,6 +153,11 @@ func (o *Option) GetTSOClientRPCConcurrency() int { return o.dynamicOptions[TSOClientRPCConcurrency].Load().(int) } +// GetRegionMetadataCircuitBreakerSettings gets circuit breaker settings for PD region metadata calls. +func (o *Option) GetRegionMetadataCircuitBreakerSettings() cb.Settings { + return o.dynamicOptions[RegionMetadataCircuitBreakerSettings].Load().(cb.Settings) +} + // ClientOption configures client. type ClientOption func(*Option) @@ -201,6 +212,13 @@ func WithInitMetricsOption(initMetrics bool) ClientOption { } } +// WithRegionMetaCircuitBreaker configures the client with circuit breaker for region meta calls +func WithRegionMetaCircuitBreaker(config cb.Settings) ClientOption { + return func(op *Option) { + op.RegionMetaCircuitBreakerSettings = config + } +} + // WithBackoffer configures the client with backoffer. func WithBackoffer(bo *retry.Backoffer) ClientOption { return func(op *Option) { From 0b757ce255ffc91dcf4f2a543b54005e52bb1325 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Tue, 10 Dec 2024 18:45:39 +0800 Subject: [PATCH 02/33] cluster: fix tso fallback due raft cluster did not stop tso service (part2) (#8885) ref tikv/pd#8477, close tikv/pd#8889 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- server/cluster/cluster.go | 30 ++++++++- server/server.go | 10 ++- tests/server/api/api_test.go | 2 +- tests/server/cluster/cluster_test.go | 99 ++++++++++++++++++++++++++-- 4 files changed, 131 insertions(+), 10 deletions(-) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 9b4630964b9..10e9bf7ff1a 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -315,7 +315,7 @@ func (c *RaftCluster) InitCluster( } // Start starts a cluster. -func (c *RaftCluster) Start(s Server) error { +func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { c.Lock() defer c.Unlock() @@ -324,11 +324,29 @@ func (c *RaftCluster) Start(s Server) error { return nil } c.isAPIServiceMode = s.IsAPIServiceMode() - err := c.InitCluster(s.GetAllocator(), s.GetPersistOptions(), s.GetHBStreams(), s.GetKeyspaceGroupManager()) + err = c.InitCluster(s.GetAllocator(), s.GetPersistOptions(), s.GetHBStreams(), s.GetKeyspaceGroupManager()) if err != nil { return err } - c.checkTSOService() + // We should not manage tso service when bootstrap try to start raft cluster. + // It only is controlled by leader election. + // Ref: https://github.com/tikv/pd/issues/8836 + if !bootstrap { + c.checkTSOService() + } + defer func() { + if !bootstrap && err != nil { + c.stopTSOJobsIfNeeded() + } + }() + failpoint.Inject("raftClusterReturn", func(val failpoint.Value) { + if val, ok := val.(bool); (ok && val) || !ok { + err = errors.New("raftClusterReturn") + } else { + err = nil + } + failpoint.Return(err) + }) cluster, err := c.LoadClusterInfo() if err != nil { return err @@ -2554,3 +2572,9 @@ func (c *RaftCluster) SetServiceIndependent(name string) { func (c *RaftCluster) UnsetServiceIndependent(name string) { c.independentServices.Delete(name) } + +// GetGlobalTSOAllocator return global tso allocator +// It only is used for test. +func (c *RaftCluster) GetGlobalTSOAllocator() tso.Allocator { + return c.tsoAllocator.GetAllocator() +} diff --git a/server/server.go b/server/server.go index d7bdd92d96d..3ed3c9514ff 100644 --- a/server/server.go +++ b/server/server.go @@ -758,7 +758,7 @@ func (s *Server) bootstrapCluster(req *pdpb.BootstrapRequest) (*pdpb.BootstrapRe log.Warn("flush the bootstrap region failed", errs.ZapError(err)) } - if err := s.cluster.Start(s); err != nil { + if err := s.cluster.Start(s, true); err != nil { return nil, err } @@ -776,7 +776,7 @@ func (s *Server) createRaftCluster() error { return nil } - return s.cluster.Start(s) + return s.cluster.Start(s, false) } func (s *Server) stopRaftCluster() { @@ -2097,3 +2097,9 @@ func (s *Server) GetMaxResetTSGap() time.Duration { func (s *Server) SetClient(client *clientv3.Client) { s.client = client } + +// GetGlobalTSOAllocator return global tso allocator +// It only is used for test. +func (s *Server) GetGlobalTSOAllocator() tso.Allocator { + return s.cluster.GetGlobalTSOAllocator() +} diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 14df5ff8eea..e1e4db3a26d 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -925,7 +925,7 @@ func TestSendApiWhenRestartRaftCluster(t *testing.T) { output := sendRequest(re, leader.GetAddr()+"/pd/api/v1/min-resolved-ts", http.MethodGet, http.StatusInternalServerError) re.Contains(string(output), "TiKV cluster not bootstrapped, please start TiKV first") - err = rc.Start(leader.GetServer()) + err = rc.Start(leader.GetServer(), false) re.NoError(err) rc = leader.GetRaftCluster() re.NotNil(rc) diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index df0cf7d38a3..dfdb9cb8685 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -578,7 +578,7 @@ func TestRaftClusterRestart(t *testing.T) { re.NotNil(rc) rc.Stop() - err = rc.Start(leaderServer.GetServer()) + err = rc.Start(leaderServer.GetServer(), false) re.NoError(err) rc = leaderServer.GetRaftCluster() @@ -621,7 +621,7 @@ func TestRaftClusterMultipleRestart(t *testing.T) { for range 100 { // See https://github.com/tikv/pd/issues/8543 rc.Wait() - err = rc.Start(leaderServer.GetServer()) + err = rc.Start(leaderServer.GetServer(), false) re.NoError(err) time.Sleep(time.Millisecond) rc.Stop() @@ -629,6 +629,97 @@ func TestRaftClusterMultipleRestart(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs")) } +// TestRaftClusterStartTSOJob is used to test whether tso job service is normally closed +// when raft cluster is stopped ahead of time. +// Ref: https://github.com/tikv/pd/issues/8836 +func TestRaftClusterStartTSOJob(t *testing.T) { + re := require.New(t) + name := "pd1" + // case 1: normal start + ctx, cancel := context.WithCancel(context.Background()) + tc, err := tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + re.NoError(tc.RunInitialServers()) + re.NotEmpty(tc.WaitLeader()) + leaderServer := tc.GetLeaderServer() + re.NotNil(leaderServer) + leaderServer.BootstrapCluster() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + tc.Destroy() + cancel() + // case 2: return ahead of time but no error when start raft cluster + re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/raftClusterReturn", `return(false)`)) + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + tc.WaitLeader() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/raftClusterReturn")) + tc.Destroy() + cancel() + // case 3: meet error when start raft cluster + re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/raftClusterReturn", `return(true)`)) + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 1, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + err = tc.RunInitialServers() + re.NoError(err) + tc.WaitLeader() + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return !allocator.IsInitialize() + }) + re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/raftClusterReturn")) + tc.Destroy() + cancel() + // case 4: multiple bootstrap in 3 pd cluster + ctx, cancel = context.WithCancel(context.Background()) + tc, err = tests.NewTestCluster(ctx, 3, func(conf *config.Config, _ string) { + conf.LeaderLease = 300 + }) + re.NoError(err) + re.NoError(tc.RunInitialServers()) + re.NotEmpty(tc.WaitLeader()) + leaderServer = tc.GetLeaderServer() + re.NotNil(leaderServer) + name = leaderServer.GetLeader().GetName() + wg := sync.WaitGroup{} + for range 3 { + wg.Add(1) + go func() { + leaderServer.BootstrapCluster() + wg.Done() + }() + } + wg.Wait() + testutil.Eventually(re, func() bool { + allocator := leaderServer.GetServer().GetGlobalTSOAllocator() + return allocator.IsInitialize() + }) + re.NoError(tc.ResignLeader()) + re.NotEmpty(tc.WaitLeader()) + testutil.Eventually(re, func() bool { + allocator := tc.GetServer(name).GetServer().GetGlobalTSOAllocator() + return !allocator.IsInitialize() + }) + tc.Destroy() + cancel() +} + func newMetaStore(storeID uint64, addr, version string, state metapb.StoreState, deployPath string) *metapb.Store { return &metapb.Store{Id: storeID, Address: addr, Version: version, State: state, DeployPath: deployPath} } @@ -1437,7 +1528,7 @@ func TestTransferLeaderForScheduler(t *testing.T) { tc.WaitLeader() leaderServer = tc.GetLeaderServer() rc1 := leaderServer.GetServer().GetRaftCluster() - rc1.Start(leaderServer.GetServer()) + rc1.Start(leaderServer.GetServer(), false) re.NoError(err) re.NotNil(rc1) // region heartbeat @@ -1457,7 +1548,7 @@ func TestTransferLeaderForScheduler(t *testing.T) { tc.WaitLeader() leaderServer = tc.GetLeaderServer() rc = leaderServer.GetServer().GetRaftCluster() - rc.Start(leaderServer.GetServer()) + rc.Start(leaderServer.GetServer(), false) re.NotNil(rc) // region heartbeat id = leaderServer.GetAllocator() From fef6424e4f14affbc20e45cf20b24196ef112aa7 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 11 Dec 2024 11:21:54 +0800 Subject: [PATCH 03/33] *: fix typo in license (#8884) ref tikv/pd#4399 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/resource_group/controller/config.go | 2 +- client/resource_group/controller/controller.go | 2 +- client/resource_group/controller/controller_test.go | 2 +- client/resource_group/controller/limiter.go | 2 +- client/resource_group/controller/limiter_test.go | 2 +- client/resource_group/controller/model.go | 2 +- client/resource_group/controller/testutil.go | 2 +- client/resource_group/controller/util.go | 2 +- pkg/mcs/resourcemanager/server/config_test.go | 2 +- pkg/mcs/resourcemanager/server/token_buckets.go | 2 +- pkg/mcs/resourcemanager/server/token_buckets_test.go | 2 +- pkg/window/counter.go | 2 +- pkg/window/counter_test.go | 2 +- pkg/window/policy.go | 2 +- pkg/window/policy_test.go | 2 +- pkg/window/reduce.go | 2 +- pkg/window/window.go | 2 +- pkg/window/window_test.go | 2 +- 18 files changed, 18 insertions(+), 18 deletions(-) diff --git a/client/resource_group/controller/config.go b/client/resource_group/controller/config.go index 96c783455bb..3008d7b6e77 100644 --- a/client/resource_group/controller/config.go +++ b/client/resource_group/controller/config.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index 83bd21b1eed..46401aad1ff 100644 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/controller_test.go b/client/resource_group/controller/controller_test.go index 882f99a6868..254df36020e 100644 --- a/client/resource_group/controller/controller_test.go +++ b/client/resource_group/controller/controller_test.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index 5d9823312ca..cd05adb7dfa 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/limiter_test.go b/client/resource_group/controller/limiter_test.go index 9afebcb3d53..24cdee0bbc3 100644 --- a/client/resource_group/controller/limiter_test.go +++ b/client/resource_group/controller/limiter_test.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/model.go b/client/resource_group/controller/model.go index 9e86de69abb..88c18dc25cf 100644 --- a/client/resource_group/controller/model.go +++ b/client/resource_group/controller/model.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/testutil.go b/client/resource_group/controller/testutil.go index 01a9c3af1fc..de71cff4d0b 100644 --- a/client/resource_group/controller/testutil.go +++ b/client/resource_group/controller/testutil.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/client/resource_group/controller/util.go b/client/resource_group/controller/util.go index e3450e0ae0d..3b491e02c8f 100644 --- a/client/resource_group/controller/util.go +++ b/client/resource_group/controller/util.go @@ -11,7 +11,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/config_test.go b/pkg/mcs/resourcemanager/server/config_test.go index ae9dfc2cad3..afe22356def 100644 --- a/pkg/mcs/resourcemanager/server/config_test.go +++ b/pkg/mcs/resourcemanager/server/config_test.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/token_buckets.go b/pkg/mcs/resourcemanager/server/token_buckets.go index e0777b419eb..f72d0dfcc88 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets.go +++ b/pkg/mcs/resourcemanager/server/token_buckets.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/mcs/resourcemanager/server/token_buckets_test.go b/pkg/mcs/resourcemanager/server/token_buckets_test.go index 8ac3ec4a3ba..3d9cbd3f628 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets_test.go +++ b/pkg/mcs/resourcemanager/server/token_buckets_test.go @@ -7,7 +7,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/counter.go b/pkg/window/counter.go index 84325cdc14b..c56c202a414 100644 --- a/pkg/window/counter.go +++ b/pkg/window/counter.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/counter_test.go b/pkg/window/counter_test.go index dc43e4fee3f..2cb25babdba 100644 --- a/pkg/window/counter_test.go +++ b/pkg/window/counter_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/policy.go b/pkg/window/policy.go index fed4fedc32a..14e33e3ee74 100644 --- a/pkg/window/policy.go +++ b/pkg/window/policy.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/policy_test.go b/pkg/window/policy_test.go index f4ae9989e19..936360ccb2b 100644 --- a/pkg/window/policy_test.go +++ b/pkg/window/policy_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/reduce.go b/pkg/window/reduce.go index 23fa87177f2..0df21ff4c4f 100644 --- a/pkg/window/reduce.go +++ b/pkg/window/reduce.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/window.go b/pkg/window/window.go index 80fb5bb5714..6d7a54131ce 100644 --- a/pkg/window/window.go +++ b/pkg/window/window.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. diff --git a/pkg/window/window_test.go b/pkg/window/window_test.go index 59fb2ee0bbb..4cedd12ae01 100644 --- a/pkg/window/window_test.go +++ b/pkg/window/window_test.go @@ -10,7 +10,7 @@ // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS,g +// distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. From fbfcdb8a4613bc1f05e0feabbc8325a90da0518a Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 11 Dec 2024 14:57:27 +0800 Subject: [PATCH 04/33] client: reduce redundant concepts in the client filenames (#8894) ref tikv/pd#8690 Reduce the redundant concepts in the client filenames. Signed-off-by: JmPotato --- client/clients/router/{router_client.go => client.go} | 0 client/clients/tso/{tso_client.go => client.go} | 0 client/clients/tso/{tso_dispatcher.go => dispatcher.go} | 0 client/clients/tso/{tso_dispatcher_test.go => dispatcher_test.go} | 0 client/clients/tso/{tso_request.go => request.go} | 0 client/clients/tso/{tso_request_test.go => request_test.go} | 0 client/clients/tso/{tso_stream.go => stream.go} | 0 client/clients/tso/{tso_stream_test.go => stream_test.go} | 0 8 files changed, 0 insertions(+), 0 deletions(-) rename client/clients/router/{router_client.go => client.go} (100%) rename client/clients/tso/{tso_client.go => client.go} (100%) rename client/clients/tso/{tso_dispatcher.go => dispatcher.go} (100%) rename client/clients/tso/{tso_dispatcher_test.go => dispatcher_test.go} (100%) rename client/clients/tso/{tso_request.go => request.go} (100%) rename client/clients/tso/{tso_request_test.go => request_test.go} (100%) rename client/clients/tso/{tso_stream.go => stream.go} (100%) rename client/clients/tso/{tso_stream_test.go => stream_test.go} (100%) diff --git a/client/clients/router/router_client.go b/client/clients/router/client.go similarity index 100% rename from client/clients/router/router_client.go rename to client/clients/router/client.go diff --git a/client/clients/tso/tso_client.go b/client/clients/tso/client.go similarity index 100% rename from client/clients/tso/tso_client.go rename to client/clients/tso/client.go diff --git a/client/clients/tso/tso_dispatcher.go b/client/clients/tso/dispatcher.go similarity index 100% rename from client/clients/tso/tso_dispatcher.go rename to client/clients/tso/dispatcher.go diff --git a/client/clients/tso/tso_dispatcher_test.go b/client/clients/tso/dispatcher_test.go similarity index 100% rename from client/clients/tso/tso_dispatcher_test.go rename to client/clients/tso/dispatcher_test.go diff --git a/client/clients/tso/tso_request.go b/client/clients/tso/request.go similarity index 100% rename from client/clients/tso/tso_request.go rename to client/clients/tso/request.go diff --git a/client/clients/tso/tso_request_test.go b/client/clients/tso/request_test.go similarity index 100% rename from client/clients/tso/tso_request_test.go rename to client/clients/tso/request_test.go diff --git a/client/clients/tso/tso_stream.go b/client/clients/tso/stream.go similarity index 100% rename from client/clients/tso/tso_stream.go rename to client/clients/tso/stream.go diff --git a/client/clients/tso/tso_stream_test.go b/client/clients/tso/stream_test.go similarity index 100% rename from client/clients/tso/tso_stream_test.go rename to client/clients/tso/stream_test.go From 5d62787565f7d238f386fd702e89a1921060f545 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Wed, 11 Dec 2024 16:17:27 +0800 Subject: [PATCH 05/33] tso/local: remove local tso completely (#8864) close tikv/pd#8802 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 16 ++--- client/clients/tso/client.go | 16 ++--- pkg/mcs/tso/server/config.go | 5 +- pkg/mcs/tso/server/config_test.go | 5 -- pkg/mcs/tso/server/server.go | 9 --- pkg/storage/endpoint/service_middleware.go | 4 +- pkg/storage/endpoint/tso.go | 6 +- pkg/tso/allocator_manager.go | 30 ---------- pkg/tso/global_allocator.go | 22 +------ pkg/tso/keyspace_group_manager.go | 6 +- pkg/tso/keyspace_group_manager_test.go | 1 - pkg/tso/testutil.go | 1 - pkg/tso/tso.go | 58 ++++--------------- pkg/utils/keypath/key_path.go | 14 ----- pkg/utils/tsoutil/tso_dispatcher.go | 14 ++--- pkg/utils/tsoutil/tso_request.go | 16 +++-- server/config/config.go | 7 +-- server/server.go | 5 -- tests/cluster.go | 5 -- .../mcs/tso/keyspace_group_manager_test.go | 2 - 20 files changed, 53 insertions(+), 189 deletions(-) diff --git a/client/client.go b/client/client.go index c271f10591d..519fd478bb3 100644 --- a/client/client.go +++ b/client/client.go @@ -509,10 +509,10 @@ func (c *client) GetTSAsync(ctx context.Context) tso.TSFuture { return c.inner.dispatchTSORequestWithRetry(ctx) } -// GetLocalTSAsync implements the TSOClient interface. -// -// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the -// parameters passed in, this method will default to returning the global TSO. +// Deprecated: the Local TSO feature has been deprecated. Regardless of the +// parameters passed, the behavior of this interface will be equivalent to +// `GetTSAsync`. If you want to use a separately deployed TSO service, +// please refer to the deployment of the TSO microservice. func (c *client) GetLocalTSAsync(ctx context.Context, _ string) tso.TSFuture { return c.GetTSAsync(ctx) } @@ -523,10 +523,10 @@ func (c *client) GetTS(ctx context.Context) (physical int64, logical int64, err return resp.Wait() } -// GetLocalTS implements the TSOClient interface. -// -// Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the -// parameters passed in, this method will default to returning the global TSO. +// Deprecated: the Local TSO feature has been deprecated. Regardless of the +// parameters passed, the behavior of this interface will be equivalent to +// `GetTS`. If you want to use a separately deployed TSO service, +// please refer to the deployment of the TSO microservice. func (c *client) GetLocalTS(ctx context.Context, _ string) (physical int64, logical int64, err error) { return c.GetTS(ctx) } diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index 9c7075fe3bb..68e2163d191 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -56,15 +56,15 @@ type Client interface { // the TSO microservice. GetMinTS(ctx context.Context) (int64, int64, error) - // GetLocalTS gets a local timestamp from PD or TSO microservice. - // - // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the - // parameters passed in, this method will default to returning the global TSO. + // Deprecated: the Local TSO feature has been deprecated. Regardless of the + // parameters passed, the behavior of this interface will be equivalent to + // `GetTS`. If you want to use a separately deployed TSO service, + // please refer to the deployment of the TSO microservice. GetLocalTS(ctx context.Context, _ string) (int64, int64, error) - // GetLocalTSAsync gets a local timestamp from PD or TSO microservice, without block the caller. - // - // Deprecated: Local TSO will be completely removed in the future. Currently, regardless of the - // parameters passed in, this method will default to returning the global TSO. + // Deprecated: the Local TSO feature has been deprecated. Regardless of the + // parameters passed, the behavior of this interface will be equivalent to + // `GetTSAsync`. If you want to use a separately deployed TSO service, + // please refer to the deployment of the TSO microservice. GetLocalTSAsync(ctx context.Context, _ string) TSFuture } diff --git a/pkg/mcs/tso/server/config.go b/pkg/mcs/tso/server/config.go index 0973042b912..2aaa54114da 100644 --- a/pkg/mcs/tso/server/config.go +++ b/pkg/mcs/tso/server/config.go @@ -64,10 +64,7 @@ type Config struct { // the primary/leader again. Etcd only supports seconds TTL, so here is second too. LeaderLease int64 `toml:"lease" json:"lease"` - // EnableLocalTSO is used to enable the Local TSO Allocator feature, - // which allows the PD server to generate Local TSO for certain DC-level transactions. - // To make this feature meaningful, user has to set the "zone" label for the PD server - // to indicate which DC this PD belongs to. + // Deprecated EnableLocalTSO bool `toml:"enable-local-tso" json:"enable-local-tso"` // TSOSaveInterval is the interval to save timestamp. diff --git a/pkg/mcs/tso/server/config_test.go b/pkg/mcs/tso/server/config_test.go index 2bd27a67492..2bafec30aa9 100644 --- a/pkg/mcs/tso/server/config_test.go +++ b/pkg/mcs/tso/server/config_test.go @@ -36,7 +36,6 @@ func TestConfigBasic(t *testing.T) { re.Equal(defaultBackendEndpoints, cfg.BackendEndpoints) re.Equal(defaultListenAddr, cfg.ListenAddr) re.Equal(constant.DefaultLeaderLease, cfg.LeaderLease) - re.False(cfg.EnableLocalTSO) re.True(cfg.EnableGRPCGateway) re.Equal(defaultTSOSaveInterval, cfg.TSOSaveInterval.Duration) re.Equal(defaultTSOUpdatePhysicalInterval, cfg.TSOUpdatePhysicalInterval.Duration) @@ -48,7 +47,6 @@ func TestConfigBasic(t *testing.T) { cfg.ListenAddr = "test-listen-addr" cfg.AdvertiseListenAddr = "test-advertise-listen-addr" cfg.LeaderLease = 123 - cfg.EnableLocalTSO = true cfg.TSOSaveInterval.Duration = time.Duration(10) * time.Second cfg.TSOUpdatePhysicalInterval.Duration = time.Duration(100) * time.Millisecond cfg.MaxResetTSGap.Duration = time.Duration(1) * time.Hour @@ -58,7 +56,6 @@ func TestConfigBasic(t *testing.T) { re.Equal("test-listen-addr", cfg.GetListenAddr()) re.Equal("test-advertise-listen-addr", cfg.GetAdvertiseListenAddr()) re.Equal(int64(123), cfg.GetLeaderLease()) - re.True(cfg.EnableLocalTSO) re.Equal(time.Duration(10)*time.Second, cfg.TSOSaveInterval.Duration) re.Equal(time.Duration(100)*time.Millisecond, cfg.TSOUpdatePhysicalInterval.Duration) re.Equal(time.Duration(1)*time.Hour, cfg.MaxResetTSGap.Duration) @@ -74,7 +71,6 @@ name = "tso-test-name" data-dir = "/var/lib/tso" enable-grpc-gateway = false lease = 123 -enable-local-tso = true tso-save-interval = "10s" tso-update-physical-interval = "100ms" max-gap-reset-ts = "1h" @@ -92,7 +88,6 @@ max-gap-reset-ts = "1h" re.Equal("test-advertise-listen-addr", cfg.GetAdvertiseListenAddr()) re.Equal("/var/lib/tso", cfg.DataDir) re.Equal(int64(123), cfg.GetLeaderLease()) - re.True(cfg.EnableLocalTSO) re.Equal(time.Duration(10)*time.Second, cfg.TSOSaveInterval.Duration) re.Equal(time.Duration(100)*time.Millisecond, cfg.TSOUpdatePhysicalInterval.Duration) re.Equal(time.Duration(1)*time.Hour, cfg.MaxResetTSGap.Duration) diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 04e81c2d48e..d2974075e94 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -272,15 +272,6 @@ func (s *Server) GetTSOAllocatorManager(keyspaceGroupID uint32) (*tso.AllocatorM return s.keyspaceGroupManager.GetAllocatorManager(keyspaceGroupID) } -// IsLocalRequest checks if the forwarded host is the current host -func (*Server) IsLocalRequest(forwardedHost string) bool { - // TODO: Check if the forwarded host is the current host. - // The logic is depending on etcd service mode -- if the TSO service - // uses the embedded etcd, check against ClientUrls; otherwise check - // against the cluster membership. - return forwardedHost == "" -} - // ValidateInternalRequest checks if server is closed, which is used to validate // the gRPC communication between TSO servers internally. // TODO: Check if the sender is from the global TSO allocator diff --git a/pkg/storage/endpoint/service_middleware.go b/pkg/storage/endpoint/service_middleware.go index 3859dab4d62..35f0606f9d0 100644 --- a/pkg/storage/endpoint/service_middleware.go +++ b/pkg/storage/endpoint/service_middleware.go @@ -29,7 +29,7 @@ type ServiceMiddlewareStorage interface { var _ ServiceMiddlewareStorage = (*StorageEndpoint)(nil) -// LoadServiceMiddlewareConfig loads service middleware config from keypath.KeyspaceGroupLocalTSPath then unmarshal it to cfg. +// LoadServiceMiddlewareConfig loads service middleware config from ServiceMiddlewarePath then unmarshal it to cfg. func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) { value, err := se.Load(keypath.ServiceMiddlewarePath) if err != nil || value == "" { @@ -42,7 +42,7 @@ func (se *StorageEndpoint) LoadServiceMiddlewareConfig(cfg any) (bool, error) { return true, nil } -// SaveServiceMiddlewareConfig stores marshallable cfg to the keypath.KeyspaceGroupLocalTSPath. +// SaveServiceMiddlewareConfig stores marshallable cfg to the ServiceMiddlewarePath. func (se *StorageEndpoint) SaveServiceMiddlewareConfig(cfg any) error { return se.saveJSON(keypath.ServiceMiddlewarePath, cfg) } diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index a656f6d2945..77841529e98 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -37,9 +37,9 @@ type TSOStorage interface { var _ TSOStorage = (*StorageEndpoint)(nil) -// LoadTimestamp will get all time windows of Local/Global TSOs from etcd and return the biggest one. -// For the Global TSO, loadTimestamp will get all Local and Global TSO time windows persisted in etcd and choose the biggest one. -// For the Local TSO, loadTimestamp will only get its own dc-location time window persisted before. +// LoadTimestamp will get all time windows of Global TSOs from etcd and return the biggest one. +// TODO: Due to local TSO is deprecated, maybe we do not need to load timestamp +// by prefix, we can just load the timestamp by the key. func (se *StorageEndpoint) LoadTimestamp(prefix string) (time.Time, error) { prefixEnd := clientv3.GetPrefixRangeEnd(prefix) keys, values, err := se.LoadRange(prefix, prefixEnd, 0) diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 8d5589143aa..65f61e819d1 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -16,8 +16,6 @@ package tso import ( "context" - "math" - "path" "runtime/trace" "strconv" "sync" @@ -43,8 +41,6 @@ const ( checkStep = time.Minute patrolStep = time.Second defaultAllocatorLeaderLease = 3 - localTSOAllocatorEtcdPrefix = "lta" - localTSOSuffixEtcdPrefix = "lts" ) var ( @@ -217,17 +213,6 @@ func (am *AllocatorManager) getGroupIDStr() string { return strconv.FormatUint(uint64(am.kgID), 10) } -// GetTimestampPath returns the timestamp path in etcd. -func (am *AllocatorManager) GetTimestampPath() string { - if am == nil { - return "" - } - - am.mu.RLock() - defer am.mu.RUnlock() - return path.Join(am.rootPath, am.mu.allocatorGroup.allocator.GetTimestampPath()) -} - // tsoAllocatorLoop is used to run the TSO Allocator updating daemon. func (am *AllocatorManager) tsoAllocatorLoop() { defer logutil.LogPanic() @@ -254,21 +239,6 @@ func (am *AllocatorManager) GetMember() ElectionMember { return am.member } -// GetSuffixBits calculates the bits of suffix sign -// by the max number of suffix so far, -// which will be used in the TSO logical part. -func (am *AllocatorManager) GetSuffixBits() int { - am.mu.RLock() - defer am.mu.RUnlock() - return CalSuffixBits(am.mu.maxSuffix) -} - -// CalSuffixBits calculates the bits of suffix by the max suffix sign. -func CalSuffixBits(maxSuffix int32) int { - // maxSuffix + 1 because we have the Global TSO holds 0 as the suffix sign - return int(math.Ceil(math.Log2(float64(maxSuffix + 1)))) -} - // AllocatorDaemon is used to update every allocator's TSO and check whether we have // any new local allocator that needs to be set up. func (am *AllocatorManager) AllocatorDaemon(ctx context.Context) { diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 740317c676a..2fe0df3e000 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -47,14 +47,6 @@ type Allocator interface { IsInitialize() bool // UpdateTSO is used to update the TSO in memory and the time window in etcd. UpdateTSO() error - // GetTimestampPath returns the timestamp path in etcd, which is: - // 1. for the default keyspace group: - // a. timestamp in /pd/{cluster_id}/timestamp - // b. lta/{dc-location}/timestamp in /pd/{cluster_id}/lta/{dc-location}/timestamp - // 1. for the non-default keyspace groups: - // a. {group}/gts/timestamp in /ms/{cluster_id}/tso/{group}/gta/timestamp - // b. {group}/lts/{dc-location}/timestamp in /ms/{cluster_id}/tso/{group}/lta/{dc-location}/timestamp - GetTimestampPath() string // SetTSO sets the physical part with given TSO. It's mainly used for BR restore. // Cannot set the TSO smaller than now in any case. // if ignoreSmaller=true, if input ts is smaller than current, ignore silently, else return error @@ -68,6 +60,8 @@ type Allocator interface { } // GlobalTSOAllocator is the global single point TSO allocator. +// TODO: Local TSO allocator is deprecated now, we can update the name to +// TSOAllocator and remove the `Global` concept. type GlobalTSOAllocator struct { ctx context.Context cancel context.CancelFunc @@ -132,19 +126,9 @@ func (gta *GlobalTSOAllocator) getGroupID() uint32 { return gta.am.getGroupID() } -// GetTimestampPath returns the timestamp path in etcd. -func (gta *GlobalTSOAllocator) GetTimestampPath() string { - if gta == nil || gta.timestampOracle == nil { - return "" - } - return gta.timestampOracle.GetTimestampPath() -} - // Initialize will initialize the created global TSO allocator. func (gta *GlobalTSOAllocator) Initialize(int) error { gta.tsoAllocatorRoleGauge.Set(1) - // The suffix of a Global TSO should always be 0. - gta.timestampOracle.suffix = 0 return gta.timestampOracle.SyncTimestamp() } @@ -175,7 +159,7 @@ func (gta *GlobalTSOAllocator) GenerateTSO(ctx context.Context, count uint32) (p return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested pd %s of cluster", errs.NotLeaderErr)) } - return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count, 0) + return gta.timestampOracle.getTS(ctx, gta.member.GetLeadership(), count) } // Reset is used to reset the TSO allocator. diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 86b43d0de45..bb5fb4587f7 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -66,7 +66,7 @@ const ( type state struct { syncutil.RWMutex // ams stores the allocator managers of the keyspace groups. Each keyspace group is - // assigned with an allocator manager managing its global/local tso allocators. + // assigned with an allocator manager managing its global tso allocators. // Use a fixed size array to maximize the efficiency of concurrent access to // different keyspace groups for tso service. ams [constant.MaxKeyspaceGroupCountInUse]*AllocatorManager @@ -790,8 +790,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro am := NewAllocatorManager(kgm.ctx, group.ID, participant, tsRootPath, storage, kgm.cfg) am.startGlobalAllocatorLoop() log.Info("created allocator manager", - zap.Uint32("keyspace-group-id", group.ID), - zap.String("timestamp-path", am.GetTimestampPath())) + zap.Uint32("keyspace-group-id", group.ID)) kgm.Lock() group.KeyspaceLookupTable = make(map[uint32]struct{}) for _, kid := range group.Keyspaces { @@ -1517,7 +1516,6 @@ func (kgm *KeyspaceGroupManager) deletedGroupCleaner() { log.Info("delete the keyspace group tso key", zap.Uint32("keyspace-group-id", groupID)) // Clean up the remaining TSO keys. - // TODO: support the Local TSO Allocator clean up. err := kgm.tsoSvcStorage.DeleteTimestamp( keypath.TimestampPath( keypath.KeyspaceGroupGlobalTSPath(groupID), diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index dea0b00f4f0..be3d53785cd 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -87,7 +87,6 @@ func (suite *keyspaceGroupManagerTestSuite) createConfig() *TestServiceConfig { ListenAddr: addr, AdvertiseListenAddr: addr, LeaderLease: constant.DefaultLeaderLease, - LocalTSOEnabled: false, TSOUpdatePhysicalInterval: 50 * time.Millisecond, TSOSaveInterval: time.Duration(constant.DefaultLeaderLease) * time.Second, MaxResetTSGap: time.Hour * 24, diff --git a/pkg/tso/testutil.go b/pkg/tso/testutil.go index e3d04f55813..336d1414d98 100644 --- a/pkg/tso/testutil.go +++ b/pkg/tso/testutil.go @@ -29,7 +29,6 @@ type TestServiceConfig struct { ListenAddr string // Address the service listens on. AdvertiseListenAddr string // Address the service advertises to the clients. LeaderLease int64 // Leader lease. - LocalTSOEnabled bool // Whether local TSO is enabled. TSOUpdatePhysicalInterval time.Duration // Interval to update TSO in physical storage. TSOSaveInterval time.Duration // Interval to save TSO to physical storage. MaxResetTSGap time.Duration // Maximum gap to reset TSO. diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 38a4c989093..0210c98626b 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -43,8 +43,6 @@ const ( // When a TSO's logical time reaches this limit, // the physical time will be forced to increase. maxLogical = int64(1 << 18) - // MaxSuffixBits indicates the max number of suffix bits. - MaxSuffixBits = 4 // jetLagWarningThreshold is the warning threshold of jetLag in `timestampOracle.UpdateTimestamp`. // In case of small `updatePhysicalInterval`, the `3 * updatePhysicalInterval` would also is small, // and trigger unnecessary warnings about clock offset. @@ -55,9 +53,8 @@ const ( // tsoObject is used to store the current TSO in memory with a RWMutex lock. type tsoObject struct { syncutil.RWMutex - physical time.Time - logical int64 - updateTime time.Time + physical time.Time + logical int64 } // timestampOracle is used to maintain the logic of TSO. @@ -75,7 +72,6 @@ type timestampOracle struct { tsoMux *tsoObject // last timestamp window stored in etcd lastSavedTime atomic.Value // stored as time.Time - suffix int // pre-initialized metrics metrics *tsoMetrics @@ -92,7 +88,6 @@ func (t *timestampOracle) setTSOPhysical(next time.Time, force bool) { if typeutil.SubTSOPhysicalByWallClock(next, t.tsoMux.physical) > 0 { t.tsoMux.physical = next t.tsoMux.logical = 0 - t.tsoMux.updateTime = time.Now() } } @@ -106,23 +101,17 @@ func (t *timestampOracle) getTSO() (time.Time, int64) { } // generateTSO will add the TSO's logical part with the given count and returns the new TSO result. -func (t *timestampOracle) generateTSO(ctx context.Context, count int64, suffixBits int) (physical int64, logical int64, lastUpdateTime time.Time) { +func (t *timestampOracle) generateTSO(ctx context.Context, count int64) (physical int64, logical int64) { defer trace.StartRegion(ctx, "timestampOracle.generateTSO").End() t.tsoMux.Lock() defer t.tsoMux.Unlock() if t.tsoMux.physical == typeutil.ZeroTime { - return 0, 0, typeutil.ZeroTime + return 0, 0 } physical = t.tsoMux.physical.UnixNano() / int64(time.Millisecond) t.tsoMux.logical += count logical = t.tsoMux.logical - if suffixBits > 0 && t.suffix >= 0 { - logical = t.calibrateLogical(logical, suffixBits) - } - // Return the last update time - lastUpdateTime = t.tsoMux.updateTime - t.tsoMux.updateTime = time.Now() - return physical, logical, lastUpdateTime + return physical, logical } func (t *timestampOracle) getLastSavedTime() time.Time { @@ -133,28 +122,6 @@ func (t *timestampOracle) getLastSavedTime() time.Time { return last.(time.Time) } -// Because the Local TSO in each Local TSO Allocator is independent, so they are possible -// to be the same at sometimes, to avoid this case, we need to use the logical part of the -// Local TSO to do some differentiating work. -// For example, we have three DCs: dc-1, dc-2 and dc-3. The bits of suffix is defined by -// the const suffixBits. Then, for dc-2, the suffix may be 1 because it's persisted -// in etcd with the value of 1. -// Once we get a normal TSO like this (18 bits): xxxxxxxxxxxxxxxxxx. We will make the TSO's -// low bits of logical part from each DC looks like: -// -// global: xxxxxxxxxx00000000 -// dc-1: xxxxxxxxxx00000001 -// dc-2: xxxxxxxxxx00000010 -// dc-3: xxxxxxxxxx00000011 -func (t *timestampOracle) calibrateLogical(rawLogical int64, suffixBits int) int64 { - return rawLogical< 0)) @@ -209,7 +176,7 @@ func (t *timestampOracle) SyncTimestamp() error { }) save := next.Add(t.saveInterval) start := time.Now() - if err = t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err = t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { t.metrics.errSaveSyncTSEvent.Inc() return err } @@ -277,7 +244,7 @@ func (t *timestampOracle) resetUserTimestamp(leadership *election.Leadership, ts if typeutil.SubRealTimeByWallClock(t.getLastSavedTime(), nextPhysical) <= UpdateTimestampGuard { save := nextPhysical.Add(t.saveInterval) start := time.Now() - if err := t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err := t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { t.metrics.errSaveResetTSEvent.Inc() return err } @@ -287,7 +254,6 @@ func (t *timestampOracle) resetUserTimestamp(leadership *election.Leadership, ts // save into memory only if nextPhysical or nextLogical is greater. t.tsoMux.physical = nextPhysical t.tsoMux.logical = int64(nextLogical) - t.tsoMux.updateTime = time.Now() t.metrics.resetTSOOKEvent.Inc() return nil } @@ -361,10 +327,10 @@ func (t *timestampOracle) UpdateTimestamp() error { if typeutil.SubRealTimeByWallClock(t.getLastSavedTime(), next) <= UpdateTimestampGuard { save := next.Add(t.saveInterval) start := time.Now() - if err := t.storage.SaveTimestamp(t.GetTimestampPath(), save); err != nil { + if err := t.storage.SaveTimestamp(keypath.TimestampPath(t.tsPath), save); err != nil { log.Warn("save timestamp failed", logutil.CondUint32("keyspace-group-id", t.keyspaceGroupID, t.keyspaceGroupID > 0), - zap.String("timestamp-path", t.GetTimestampPath()), + zap.String("timestamp-path", keypath.TimestampPath(t.tsPath)), zap.Error(err)) t.metrics.errSaveUpdateTSEvent.Inc() return err @@ -381,7 +347,7 @@ func (t *timestampOracle) UpdateTimestamp() error { var maxRetryCount = 10 // getTS is used to get a timestamp. -func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leadership, count uint32, suffixBits int) (pdpb.Timestamp, error) { +func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leadership, count uint32) (pdpb.Timestamp, error) { defer trace.StartRegion(ctx, "timestampOracle.getTS").End() var resp pdpb.Timestamp if count == 0 { @@ -399,7 +365,7 @@ func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leader return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory isn't initialized") } // Get a new TSO result with the given count - resp.Physical, resp.Logical, _ = t.generateTSO(ctx, int64(count), suffixBits) + resp.Physical, resp.Logical = t.generateTSO(ctx, int64(count)) if resp.GetPhysical() == 0 { return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs("timestamp in memory has been reset") } @@ -416,7 +382,6 @@ func (t *timestampOracle) getTS(ctx context.Context, leadership *election.Leader if !leadership.Check() { return pdpb.Timestamp{}, errs.ErrGenerateTimestamp.FastGenByArgs(fmt.Sprintf("requested %s anymore", errs.NotLeaderErr)) } - resp.SuffixBits = uint32(suffixBits) return resp, nil } t.metrics.exceededMaxRetryEvent.Inc() @@ -430,6 +395,5 @@ func (t *timestampOracle) ResetTimestamp() { log.Info("reset the timestamp in memory", logutil.CondUint32("keyspace-group-id", t.keyspaceGroupID, t.keyspaceGroupID > 0)) t.tsoMux.physical = typeutil.ZeroTime t.tsoMux.logical = 0 - t.tsoMux.updateTime = typeutil.ZeroTime t.lastSavedTime.Store(typeutil.ZeroTime) } diff --git a/pkg/utils/keypath/key_path.go b/pkg/utils/keypath/key_path.go index 4d59fafe16f..1a56ad3330a 100644 --- a/pkg/utils/keypath/key_path.go +++ b/pkg/utils/keypath/key_path.go @@ -370,20 +370,6 @@ func TimestampPath(tsPath string) string { return path.Join(tsPath, TimestampKey) } -// FullTimestampPath returns the full timestamp path. -// 1. for the default keyspace group: -// /pd/{cluster_id}/timestamp -// 2. for the non-default keyspace groups: -// /ms/{cluster_id}/tso/{group}/gta/timestamp -func FullTimestampPath(groupID uint32) string { - rootPath := TSOSvcRootPath() - tsPath := TimestampPath(KeyspaceGroupGlobalTSPath(groupID)) - if groupID == constant.DefaultKeyspaceGroupID { - rootPath = LegacyRootPath() - } - return path.Join(rootPath, tsPath) -} - const ( registryKey = "registry" ) diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index c4aa96274e1..86afe75e6c5 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -171,23 +171,23 @@ func (s *TSODispatcher) processRequests(forwardStream stream, requests []Request s.tsoProxyBatchSize.Observe(float64(count)) // Split the response ts := resp.GetTimestamp() - physical, logical, suffixBits := ts.GetPhysical(), ts.GetLogical(), ts.GetSuffixBits() + physical, logical := ts.GetPhysical(), ts.GetLogical() // `logical` is the largest ts's logical part here, we need to do the subtracting before we finish each TSO request. // This is different from the logic of client batch, for example, if we have a largest ts whose logical part is 10, // count is 5, then the splitting results should be 5 and 10. - firstLogical := addLogical(logical, -int64(count), suffixBits) - return s.finishRequest(requests, physical, firstLogical, suffixBits) + firstLogical := addLogical(logical, -int64(count)) + return s.finishRequest(requests, physical, firstLogical) } // Because of the suffix, we need to shift the count before we add it to the logical part. -func addLogical(logical, count int64, suffixBits uint32) int64 { - return logical + count< Date: Tue, 17 Dec 2024 14:49:54 +0800 Subject: [PATCH 06/33] client/batch: allow tokenCh of batch controller to be nil (#8903) ref tikv/pd#8690 Allow `tokenCh` of batch controller be nil. Signed-off-by: JmPotato --- client/pkg/batch/batch_controller.go | 37 +++++++++----- client/pkg/batch/batch_controller_test.go | 61 ++++++++++++++++++++++- 2 files changed, 83 insertions(+), 15 deletions(-) diff --git a/client/pkg/batch/batch_controller.go b/client/pkg/batch/batch_controller.go index 32f0aaba1ae..322502b754a 100644 --- a/client/pkg/batch/batch_controller.go +++ b/client/pkg/batch/batch_controller.go @@ -60,13 +60,18 @@ func NewController[T any](maxBatchSize int, finisher FinisherFunc[T], bestBatchO // It returns nil error if everything goes well, otherwise a non-nil error which means we should stop the service. // It's guaranteed that if this function failed after collecting some requests, then these requests will be cancelled // when the function returns, so the caller don't need to clear them manually. +// `tokenCh` is an optional parameter: +// - If it's nil, the batching process will not wait for the token to arrive to continue. +// - If it's not nil, the batching process will wait for a token to arrive before continuing. +// The token will be given back if any error occurs, otherwise it's the caller's responsibility +// to decide when to recycle the signal. func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-chan T, tokenCh chan struct{}, maxBatchWaitInterval time.Duration) (errRet error) { var tokenAcquired bool defer func() { if errRet != nil { // Something went wrong when collecting a batch of requests. Release the token and cancel collected requests // if any. - if tokenAcquired { + if tokenAcquired && tokenCh != nil { tokenCh <- struct{}{} } bc.FinishCollectedRequests(bc.finisher, errRet) @@ -80,6 +85,9 @@ func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-c // 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 bc.collectedRequestCount >= bc.maxBatchSize && !tokenAcquired { + if tokenCh == nil { + return nil + } select { case <-ctx.Done(): return ctx.Err() @@ -88,20 +96,23 @@ func (bc *Controller[T]) FetchPendingRequests(ctx context.Context, requestCh <-c } } - select { - case <-ctx.Done(): - return ctx.Err() - case req := <-requestCh: - // Start to batch when the first request arrives. - bc.pushRequest(req) - // A request arrives but the token is not ready yet. Continue waiting, and also allowing collecting the next - // request if it arrives. - continue - case <-tokenCh: - tokenAcquired = true + if tokenCh != nil { + select { + case <-ctx.Done(): + return ctx.Err() + case req := <-requestCh: + // Start to batch when the first request arrives. + bc.pushRequest(req) + // A request arrives but the token is not ready yet. Continue waiting, and also allowing collecting the next + // request if it arrives. + continue + case <-tokenCh: + tokenAcquired = true + } } - // The token is ready. If the first request didn't arrive, wait for it. + // After the token is ready or it's working without token, + // wait for the first request to arrive. if bc.collectedRequestCount == 0 { select { case <-ctx.Done(): diff --git a/client/pkg/batch/batch_controller_test.go b/client/pkg/batch/batch_controller_test.go index 7c9ffa6944f..92aef14bd35 100644 --- a/client/pkg/batch/batch_controller_test.go +++ b/client/pkg/batch/batch_controller_test.go @@ -21,9 +21,11 @@ import ( "github.com/stretchr/testify/require" ) +const testMaxBatchSize = 20 + func TestAdjustBestBatchSize(t *testing.T) { re := require.New(t) - bc := NewController[int](20, nil, nil) + bc := NewController[int](testMaxBatchSize, nil, nil) re.Equal(defaultBestBatchSize, bc.bestBatchSize) bc.AdjustBestBatchSize() re.Equal(defaultBestBatchSize-1, bc.bestBatchSize) @@ -52,7 +54,7 @@ type testRequest struct { func TestFinishCollectedRequests(t *testing.T) { re := require.New(t) - bc := NewController[*testRequest](20, nil, nil) + bc := NewController[*testRequest](testMaxBatchSize, nil, nil) // Finish with zero request count. re.Zero(bc.collectedRequestCount) bc.FinishCollectedRequests(nil, nil) @@ -81,3 +83,58 @@ func TestFinishCollectedRequests(t *testing.T) { re.Equal(context.Canceled, requests[i].err) } } + +func TestFetchPendingRequests(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + re := require.New(t) + bc := NewController[int](testMaxBatchSize, nil, nil) + requestCh := make(chan int, testMaxBatchSize+1) + // Fetch a nil `tokenCh`. + requestCh <- 1 + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Empty(requestCh) + re.Equal(1, bc.collectedRequestCount) + // Fetch a nil `tokenCh` with max batch size. + for i := range testMaxBatchSize { + requestCh <- i + } + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Empty(requestCh) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Fetch a nil `tokenCh` with max batch size + 1. + for i := range testMaxBatchSize + 1 { + requestCh <- i + } + re.NoError(bc.FetchPendingRequests(ctx, requestCh, nil, 0)) + re.Len(requestCh, 1) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Drain the requestCh. + <-requestCh + // Fetch a non-nil `tokenCh`. + tokenCh := make(chan struct{}, 1) + requestCh <- 1 + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Empty(requestCh) + re.Equal(1, bc.collectedRequestCount) + // Fetch a non-nil `tokenCh` with max batch size. + for i := range testMaxBatchSize { + requestCh <- i + } + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Empty(requestCh) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Fetch a non-nil `tokenCh` with max batch size + 1. + for i := range testMaxBatchSize + 1 { + requestCh <- i + } + tokenCh <- struct{}{} + re.NoError(bc.FetchPendingRequests(ctx, requestCh, tokenCh, 0)) + re.Len(requestCh, 1) + re.Equal(testMaxBatchSize, bc.collectedRequestCount) + // Drain the requestCh. + <-requestCh +} From e907aac6fc34cab0c2b0762ec95ab8355ca532fd Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 17 Dec 2024 15:23:16 +0800 Subject: [PATCH 07/33] *: enable gci (#8925) ref tikv/pd#4322 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- .golangci.yml | 8 ++++++++ client/circuitbreaker/circuit_breaker.go | 6 +++--- client/circuitbreaker/circuit_breaker_test.go | 4 ++-- client/client.go | 9 +++++---- client/client_test.go | 3 ++- client/clients/metastorage/client.go | 1 + client/clients/router/client.go | 1 + client/clients/tso/client.go | 12 +++++++----- client/clients/tso/dispatcher.go | 4 +++- client/clients/tso/dispatcher_test.go | 8 +++++--- client/clients/tso/request.go | 1 + client/clients/tso/request_test.go | 3 ++- client/clients/tso/stream.go | 8 +++++--- client/clients/tso/stream_test.go | 8 +++++--- client/errs/errs.go | 3 ++- client/gc_client.go | 4 +++- client/http/client.go | 6 ++++-- client/http/client_test.go | 1 + client/http/interface.go | 1 + client/http/request_info.go | 3 ++- client/http/types.go | 1 + client/inner_client.go | 7 ++++--- client/keyspace_client.go | 2 ++ client/meta_storage_client.go | 4 +++- client/opt/option.go | 7 ++++--- client/opt/option_test.go | 1 + client/pkg/retry/backoff.go | 3 ++- client/pkg/retry/backoff_test.go | 3 ++- client/pkg/utils/grpcutil/grpcutil.go | 12 +++++++----- client/pkg/utils/testutil/check_env_linux.go | 3 ++- client/pkg/utils/testutil/tempurl.go | 3 ++- client/pkg/utils/tlsutil/tlsconfig.go | 1 + client/pkg/utils/tlsutil/url.go | 3 ++- .../resource_group/controller/controller.go | 8 +++++--- .../controller/controller_test.go | 6 ++++-- client/resource_group/controller/limiter.go | 6 ++++-- .../resource_group/controller/model_test.go | 3 ++- client/resource_manager_client.go | 4 +++- .../servicediscovery/pd_service_discovery.go | 12 +++++++----- .../pd_service_discovery_test.go | 16 +++++++++------- .../servicediscovery/tso_service_discovery.go | 6 ++++-- cmd/pd-server/main.go | 6 ++++-- pkg/audit/audit.go | 6 ++++-- pkg/audit/audit_test.go | 1 + pkg/autoscaling/calculation.go | 8 +++++--- pkg/autoscaling/calculation_test.go | 1 + pkg/autoscaling/handler.go | 3 ++- pkg/autoscaling/prometheus.go | 8 +++++--- pkg/autoscaling/service.go | 5 +++-- pkg/autoscaling/types.go | 3 ++- pkg/cache/cache_test.go | 1 + pkg/cache/ttl.go | 4 +++- pkg/cgroup/cgmon.go | 6 ++++-- pkg/cgroup/cgroup.go | 3 ++- pkg/core/factory_test.go | 2 ++ pkg/core/region.go | 4 +++- pkg/core/region_test.go | 4 +++- pkg/core/region_tree.go | 4 +++- pkg/core/region_tree_test.go | 3 ++- pkg/core/store.go | 4 +++- pkg/core/store_option.go | 1 + pkg/core/store_stats.go | 1 + pkg/core/store_stats_test.go | 3 ++- pkg/core/store_test.go | 4 +++- pkg/core/storelimit/limit_test.go | 1 + pkg/dashboard/adapter/manager.go | 1 + pkg/dashboard/adapter/redirector.go | 1 + pkg/dashboard/distroutil/distro.go | 3 ++- pkg/dashboard/keyvisual/input/core.go | 4 +++- .../uiserver/embedded_assets_rewriter.go | 1 + pkg/election/leadership.go | 8 +++++--- pkg/election/leadership_test.go | 8 +++++--- pkg/election/lease.go | 6 ++++-- pkg/election/lease_test.go | 1 + pkg/encryption/config.go | 1 + pkg/encryption/config_test.go | 1 + pkg/encryption/crypter.go | 1 + pkg/encryption/crypter_test.go | 3 ++- pkg/encryption/key_manager.go | 8 +++++--- pkg/encryption/key_manager_test.go | 6 ++++-- pkg/encryption/kms.go | 2 ++ pkg/encryption/master_key.go | 1 + pkg/encryption/master_key_test.go | 3 ++- pkg/encryption/region_crypter.go | 1 + pkg/encryption/region_crypter_test.go | 3 ++- pkg/errs/errs.go | 3 ++- pkg/errs/errs_test.go | 5 +++-- pkg/gc/safepoint.go | 1 + pkg/gc/safepoint_test.go | 1 + pkg/gc/safepoint_v2.go | 4 +++- pkg/gctuner/memory_limit_tuner.go | 4 +++- pkg/gctuner/memory_limit_tuner_test.go | 4 +++- pkg/gctuner/tuner.go | 4 +++- pkg/id/id.go | 8 +++++--- pkg/id/id_test.go | 1 + pkg/keyspace/keyspace.go | 4 +++- pkg/keyspace/keyspace_test.go | 6 ++++-- pkg/keyspace/tso_keyspace_group.go | 8 +++++--- pkg/keyspace/tso_keyspace_group_test.go | 1 + pkg/keyspace/util.go | 1 + pkg/keyspace/util_test.go | 1 + pkg/mcs/discovery/discover.go | 6 ++++-- pkg/mcs/discovery/discover_test.go | 1 + pkg/mcs/discovery/register.go | 6 ++++-- pkg/mcs/discovery/register_test.go | 5 +++-- pkg/mcs/discovery/registry_entry.go | 3 ++- pkg/mcs/metastorage/server/grpc_service.go | 12 +++++++----- pkg/mcs/metastorage/server/manager.go | 6 ++++-- pkg/mcs/registry/registry.go | 6 ++++-- pkg/mcs/resourcemanager/server/apis/v1/api.go | 2 ++ pkg/mcs/resourcemanager/server/config.go | 6 ++++-- .../resourcemanager/server/grpc_service.go | 10 ++++++---- pkg/mcs/resourcemanager/server/manager.go | 6 ++++-- .../resourcemanager/server/metrics_test.go | 3 ++- .../resourcemanager/server/resource_group.go | 4 +++- .../server/resource_group_test.go | 3 ++- .../resourcemanager/server/token_buckets.go | 3 ++- .../server/token_buckets_test.go | 3 ++- pkg/mcs/scheduling/server/apis/v1/api.go | 4 +++- pkg/mcs/scheduling/server/cluster.go | 4 +++- pkg/mcs/scheduling/server/config/config.go | 6 ++++-- pkg/mcs/scheduling/server/config/watcher.go | 8 +++++--- pkg/mcs/scheduling/server/grpc_service.go | 10 ++++++---- pkg/mcs/scheduling/server/meta/watcher.go | 8 +++++--- pkg/mcs/scheduling/server/rule/watcher.go | 8 +++++--- .../scheduling/server/rule/watcher_test.go | 5 +++-- pkg/mcs/scheduling/server/server.go | 8 +++++--- pkg/mcs/scheduling/server/testutil.go | 4 +++- pkg/mcs/server/server.go | 6 ++++-- pkg/mcs/tso/server/apis/v1/api.go | 6 ++++-- pkg/mcs/tso/server/config.go | 6 ++++-- pkg/mcs/tso/server/config_test.go | 1 + pkg/mcs/tso/server/grpc_service.go | 8 +++++--- pkg/mcs/tso/server/server.go | 12 +++++++----- pkg/mcs/tso/server/testutil.go | 3 ++- pkg/mcs/utils/expected_primary.go | 6 ++++-- pkg/mcs/utils/util.go | 16 +++++++++------- pkg/member/member.go | 8 +++++--- pkg/member/participant.go | 6 ++++-- pkg/memory/meminfo.go | 8 +++++--- pkg/mock/mockcluster/mockcluster.go | 2 ++ pkg/mock/mockhbstream/mockhbstream.go | 1 + pkg/mock/mockhbstream/mockhbstream_test.go | 4 +++- pkg/mock/mockserver/mockserver.go | 1 + pkg/ratelimit/controller_test.go | 3 ++- pkg/ratelimit/limiter.go | 3 ++- pkg/ratelimit/limiter_test.go | 3 ++- pkg/ratelimit/ratelimiter.go | 3 ++- pkg/ratelimit/runner.go | 3 ++- pkg/replication/replication_mode.go | 4 +++- pkg/replication/replication_mode_test.go | 4 +++- pkg/response/region.go | 2 ++ pkg/response/region_test.go | 3 ++- pkg/response/store.go | 1 + pkg/schedule/checker/checker_controller.go | 4 +++- pkg/schedule/checker/joint_state_checker.go | 1 + .../checker/joint_state_checker_test.go | 4 +++- pkg/schedule/checker/learner_checker.go | 1 + pkg/schedule/checker/learner_checker_test.go | 4 +++- pkg/schedule/checker/merge_checker.go | 1 + pkg/schedule/checker/merge_checker_test.go | 6 ++++-- .../checker/priority_inspector_test.go | 1 + pkg/schedule/checker/replica_checker.go | 4 +++- pkg/schedule/checker/replica_checker_test.go | 6 ++++-- pkg/schedule/checker/replica_strategy.go | 4 +++- pkg/schedule/checker/rule_checker.go | 4 +++- pkg/schedule/checker/rule_checker_test.go | 5 +++-- pkg/schedule/checker/split_checker.go | 1 + pkg/schedule/checker/split_checker_test.go | 1 + pkg/schedule/config/config.go | 1 + pkg/schedule/config/config_provider.go | 2 ++ pkg/schedule/config/store_config.go | 4 +++- pkg/schedule/config/util_test.go | 3 ++- pkg/schedule/coordinator.go | 4 +++- pkg/schedule/filter/candidates_test.go | 4 +++- pkg/schedule/filter/counter_test.go | 1 + pkg/schedule/filter/filters.go | 4 +++- pkg/schedule/filter/filters_test.go | 4 +++- pkg/schedule/filter/healthy_test.go | 4 +++- pkg/schedule/filter/region_filters_test.go | 4 +++- pkg/schedule/handler/handler.go | 4 +++- pkg/schedule/hbstream/heartbeat_streams.go | 4 +++- pkg/schedule/labeler/labeler.go | 4 +++- pkg/schedule/labeler/labeler_test.go | 4 +++- pkg/schedule/labeler/rules.go | 4 +++- pkg/schedule/operator/builder.go | 1 + pkg/schedule/operator/builder_test.go | 4 +++- pkg/schedule/operator/create_operator.go | 4 +++- pkg/schedule/operator/create_operator_test.go | 6 ++++-- pkg/schedule/operator/metrics.go | 1 + pkg/schedule/operator/operator.go | 4 +++- pkg/schedule/operator/operator_controller.go | 4 +++- .../operator/operator_controller_test.go | 6 ++++-- pkg/schedule/operator/operator_test.go | 4 +++- pkg/schedule/operator/step.go | 4 +++- pkg/schedule/operator/step_test.go | 4 +++- pkg/schedule/operator/test_util.go | 1 + .../operator/waiting_operator_test.go | 4 +++- pkg/schedule/placement/fit.go | 1 + pkg/schedule/placement/fit_region_test.go | 1 + pkg/schedule/placement/fit_test.go | 4 +++- .../placement/label_constraint_test.go | 1 + pkg/schedule/placement/region_rule_cache.go | 1 + .../placement/region_rule_cache_test.go | 4 +++- pkg/schedule/placement/rule_list.go | 1 + pkg/schedule/placement/rule_manager.go | 6 ++++-- pkg/schedule/placement/rule_manager_test.go | 4 +++- pkg/schedule/plan/balance_plan_test.go | 4 +++- pkg/schedule/plugin_interface.go | 4 +++- pkg/schedule/prepare_checker.go | 4 +++- pkg/schedule/scatter/region_scatterer.go | 4 +++- pkg/schedule/scatter/region_scatterer_test.go | 4 +++- .../schedulers/balance_benchmark_test.go | 4 +++- pkg/schedule/schedulers/balance_leader.go | 6 ++++-- .../schedulers/balance_leader_test.go | 4 +++- pkg/schedule/schedulers/balance_region.go | 4 +++- .../schedulers/balance_region_test.go | 4 +++- pkg/schedule/schedulers/balance_witness.go | 8 +++++--- .../schedulers/balance_witness_test.go | 1 + pkg/schedule/schedulers/base_scheduler.go | 1 + pkg/schedule/schedulers/config.go | 4 +++- pkg/schedule/schedulers/config_test.go | 1 + pkg/schedule/schedulers/evict_leader.go | 6 ++++-- pkg/schedule/schedulers/evict_leader_test.go | 4 +++- pkg/schedule/schedulers/evict_slow_store.go | 6 ++++-- .../schedulers/evict_slow_store_test.go | 4 +++- pkg/schedule/schedulers/evict_slow_trend.go | 6 ++++-- .../schedulers/evict_slow_trend_test.go | 4 +++- pkg/schedule/schedulers/grant_hot_region.go | 6 ++++-- pkg/schedule/schedulers/grant_leader.go | 6 ++++-- pkg/schedule/schedulers/grant_leader_test.go | 1 + pkg/schedule/schedulers/hot_region.go | 6 ++++-- pkg/schedule/schedulers/hot_region_config.go | 6 ++++-- .../schedulers/hot_region_rank_v2_test.go | 1 + pkg/schedule/schedulers/hot_region_test.go | 4 +++- pkg/schedule/schedulers/label.go | 4 +++- pkg/schedule/schedulers/metrics.go | 1 + pkg/schedule/schedulers/random_merge.go | 1 + pkg/schedule/schedulers/random_merge_test.go | 1 + pkg/schedule/schedulers/range_cluster.go | 1 + pkg/schedule/schedulers/scatter_range.go | 4 +++- pkg/schedule/schedulers/scatter_range_test.go | 4 +++- pkg/schedule/schedulers/scheduler.go | 4 +++- .../schedulers/scheduler_controller.go | 4 +++- pkg/schedule/schedulers/scheduler_test.go | 4 +++- pkg/schedule/schedulers/shuffle_hot_region.go | 6 ++++-- pkg/schedule/schedulers/shuffle_leader.go | 1 + pkg/schedule/schedulers/shuffle_region.go | 1 + .../schedulers/shuffle_region_config.go | 3 ++- pkg/schedule/schedulers/split_bucket.go | 4 +++- pkg/schedule/schedulers/split_bucket_test.go | 4 +++- .../schedulers/transfer_witness_leader.go | 1 + .../transfer_witness_leader_test.go | 4 +++- pkg/schedule/schedulers/utils.go | 4 +++- pkg/schedule/schedulers/utils_test.go | 4 +++- pkg/schedule/splitter/region_splitter.go | 4 +++- pkg/schedule/splitter/region_splitter_test.go | 1 + pkg/slice/slice_test.go | 1 + pkg/statistics/buckets/hot_bucket_cache.go | 4 +++- .../buckets/hot_bucket_cache_test.go | 3 ++- .../buckets/hot_bucket_task_test.go | 3 ++- pkg/statistics/hot_cache.go | 4 +++- pkg/statistics/hot_cache_test.go | 1 + pkg/statistics/hot_peer.go | 4 +++- pkg/statistics/hot_peer_cache.go | 4 +++- pkg/statistics/hot_peer_cache_test.go | 4 +++- pkg/statistics/region_collection.go | 4 +++- pkg/statistics/region_collection_test.go | 4 +++- pkg/statistics/store.go | 4 +++- pkg/statistics/store_collection.go | 1 + pkg/statistics/store_collection_test.go | 4 +++- pkg/statistics/store_load_test.go | 1 + pkg/statistics/store_test.go | 4 +++- pkg/statistics/utils/kind_test.go | 4 +++- pkg/storage/endpoint/cluster_id.go | 6 ++++-- pkg/storage/endpoint/cluster_id_test.go | 1 + pkg/storage/endpoint/config.go | 3 ++- pkg/storage/endpoint/gc_safe_point.go | 4 +++- pkg/storage/endpoint/keyspace.go | 4 +++- pkg/storage/endpoint/meta.go | 2 ++ pkg/storage/endpoint/resource_group.go | 1 + pkg/storage/endpoint/safepoint_v2.go | 6 ++++-- pkg/storage/endpoint/tso.go | 6 ++++-- pkg/storage/endpoint/tso_keyspace_group.go | 3 ++- pkg/storage/endpoint/util.go | 3 ++- pkg/storage/etcd_backend.go | 3 ++- pkg/storage/hot_region_storage.go | 10 ++++++---- pkg/storage/hot_region_storage_test.go | 1 + pkg/storage/keyspace_test.go | 4 +++- pkg/storage/kv/etcd_kv.go | 6 ++++-- pkg/storage/kv/kv_test.go | 3 ++- pkg/storage/kv/levedb_kv.go | 4 +++- pkg/storage/kv/mem_kv.go | 2 ++ pkg/storage/leveldb_backend.go | 4 +++- pkg/storage/leveldb_backend_test.go | 1 + pkg/storage/region_storage.go | 2 ++ pkg/storage/region_storage_test.go | 4 +++- pkg/storage/storage.go | 4 +++- pkg/storage/storage_gc_test.go | 4 +++- pkg/storage/storage_test.go | 6 ++++-- pkg/storage/storage_tso_test.go | 1 + pkg/syncer/client.go | 14 ++++++++------ pkg/syncer/client_test.go | 8 +++++--- pkg/syncer/history_buffer.go | 4 +++- pkg/syncer/history_buffer_test.go | 4 +++- pkg/syncer/server.go | 8 +++++--- pkg/systimemon/systimemon.go | 4 +++- pkg/tso/admin.go | 3 ++- pkg/tso/allocator_manager.go | 6 ++++-- pkg/tso/global_allocator.go | 6 ++++-- pkg/tso/keyspace_group_manager.go | 8 +++++--- pkg/tso/keyspace_group_manager_test.go | 10 ++++++---- pkg/tso/tso.go | 6 ++++-- pkg/tso/util_test.go | 1 + .../unsafe_recovery_controller.go | 4 +++- .../unsafe_recovery_controller_test.go | 4 +++- pkg/utils/apiutil/apiutil.go | 6 ++++-- .../apiutil/multiservicesapi/middleware.go | 4 +++- pkg/utils/apiutil/serverapi/middleware.go | 6 ++++-- pkg/utils/configutil/configutil.go | 4 +++- pkg/utils/etcdutil/etcdutil.go | 16 +++++++++------- pkg/utils/etcdutil/etcdutil_test.go | 10 ++++++---- pkg/utils/etcdutil/health_checker.go | 8 +++++--- pkg/utils/etcdutil/testutil.go | 5 +++-- pkg/utils/grpcutil/grpcutil.go | 10 ++++++---- pkg/utils/grpcutil/grpcutil_test.go | 6 ++++-- pkg/utils/logutil/log.go | 6 ++++-- pkg/utils/metricutil/metricutil.go | 4 +++- pkg/utils/metricutil/metricutil_test.go | 1 + pkg/utils/operatorutil/operator_check.go | 1 + pkg/utils/tempurl/check_env_linux.go | 2 ++ pkg/utils/tempurl/tempurl.go | 1 + pkg/utils/testutil/api_check.go | 1 + pkg/utils/testutil/testutil.go | 5 +++-- pkg/utils/tsoutil/tso_dispatcher.go | 8 +++++--- pkg/utils/tsoutil/tso_proto_factory.go | 4 +++- pkg/utils/tsoutil/tso_request.go | 4 +++- pkg/utils/typeutil/size.go | 1 + pkg/versioninfo/feature.go | 4 +++- pkg/versioninfo/versioninfo.go | 4 +++- plugin/scheduler_example/evict_leader.go | 4 +++- server/api/admin.go | 6 ++++-- server/api/admin_test.go | 4 +++- server/api/checker.go | 3 ++- server/api/cluster.go | 3 ++- server/api/cluster_test.go | 4 +++- server/api/config.go | 4 +++- server/api/diagnostic.go | 3 ++- server/api/diagnostic_test.go | 4 +++- server/api/etcd_api_test.go | 1 + server/api/health.go | 3 ++- server/api/health_test.go | 1 + server/api/hot_status.go | 3 ++- server/api/hot_status_test.go | 1 + server/api/label.go | 4 +++- server/api/label_test.go | 4 +++- server/api/log.go | 4 +++- server/api/log_test.go | 4 +++- server/api/member.go | 6 ++++-- server/api/member_test.go | 6 ++++-- server/api/middleware.go | 6 ++++-- server/api/min_resolved_ts.go | 3 ++- server/api/min_resolved_ts_test.go | 4 +++- server/api/operator.go | 3 ++- server/api/plugin_disable.go | 3 ++- server/api/pprof.go | 6 ++++-- server/api/pprof_test.go | 1 + server/api/region.go | 4 +++- server/api/region_label.go | 3 ++- server/api/region_label_test.go | 4 +++- server/api/region_test.go | 6 ++++-- server/api/replication_mode.go | 3 ++- server/api/router.go | 4 +++- server/api/rule.go | 3 ++- server/api/scheduler.go | 6 ++++-- server/api/server.go | 3 ++- server/api/server_test.go | 8 +++++--- server/api/service_gc_safepoint.go | 3 ++- server/api/service_gc_safepoint_test.go | 4 +++- server/api/service_middleware.go | 5 +++-- server/api/service_middleware_test.go | 4 +++- server/api/stats.go | 3 ++- server/api/stats_test.go | 4 +++- server/api/status.go | 3 ++- server/api/status_test.go | 1 + server/api/store.go | 4 +++- server/api/store_test.go | 6 ++++-- server/api/trend.go | 3 ++- server/api/trend_test.go | 4 +++- server/api/tso.go | 3 ++- server/api/unsafe_operation.go | 3 ++- server/api/unsafe_operation_test.go | 4 +++- server/api/version.go | 3 ++- server/api/version_test.go | 4 +++- server/apiv2/handlers/keyspace.go | 2 ++ server/apiv2/handlers/micro_service.go | 1 + server/apiv2/handlers/tso_keyspace_group.go | 2 ++ server/apiv2/middlewares/bootstrap_checker.go | 1 + server/apiv2/middlewares/redirector.go | 4 +++- server/apiv2/router.go | 1 + server/cluster/cluster.go | 6 ++++-- server/cluster/cluster_test.go | 4 +++- server/cluster/cluster_worker.go | 4 +++- server/cluster/cluster_worker_test.go | 4 +++- server/cluster/scheduling_controller.go | 1 + server/config/config.go | 10 ++++++---- server/config/config_test.go | 1 + server/config/persist_options.go | 6 ++++-- .../service_middleware_persist_options.go | 1 + server/config/util.go | 1 + server/forward.go | 10 ++++++---- server/gc_service.go | 8 +++++--- server/grpc_service.go | 14 ++++++++------ server/handler.go | 4 +++- server/join/join.go | 8 +++++--- server/join/join_test.go | 1 + server/keyspace_service.go | 6 ++++-- server/server.go | 19 +++++++++++-------- server/server_test.go | 7 ++++--- server/testutil.go | 6 ++++-- server/util.go | 6 ++++-- tests/autoscaling/autoscaling_test.go | 3 ++- tests/cluster.go | 4 +++- tests/compatibility/version_upgrade_test.go | 4 +++- tests/config.go | 1 + tests/dashboard/race_test.go | 1 + tests/dashboard/service_test.go | 3 ++- tests/integrations/client/client_test.go | 14 ++++++++------ tests/integrations/client/client_tls_test.go | 5 +++-- tests/integrations/client/gc_client_test.go | 12 +++++++----- .../integrations/client/global_config_test.go | 13 +++++++------ tests/integrations/client/http_client_test.go | 8 +++++--- tests/integrations/client/keyspace_test.go | 4 +++- .../mcs/discovery/register_test.go | 3 ++- .../mcs/keyspace/tso_keyspace_group_test.go | 4 +++- tests/integrations/mcs/members/member_test.go | 4 +++- .../resourcemanager/resource_manager_test.go | 10 ++++++---- tests/integrations/mcs/scheduling/api_test.go | 7 +++++-- .../mcs/scheduling/config_test.go | 4 +++- .../integrations/mcs/scheduling/meta_test.go | 4 +++- .../integrations/mcs/scheduling/rule_test.go | 1 + .../mcs/scheduling/server_test.go | 8 +++++--- tests/integrations/mcs/testutil.go | 1 + tests/integrations/mcs/tso/api_test.go | 4 +++- .../mcs/tso/keyspace_group_manager_test.go | 6 ++++-- tests/integrations/mcs/tso/proxy_test.go | 12 +++++++----- tests/integrations/mcs/tso/server_test.go | 14 ++++++++------ tests/integrations/realcluster/cluster.go | 3 ++- .../realcluster/cluster_id_test.go | 1 + .../integrations/realcluster/etcd_key_test.go | 1 + tests/integrations/realcluster/mock_db.go | 3 ++- .../realcluster/reboot_pd_test.go | 1 + .../realcluster/scheduler_test.go | 6 ++++-- tests/integrations/tso/client_test.go | 4 +++- tests/integrations/tso/consistency_test.go | 6 ++++-- tests/integrations/tso/server_test.go | 6 ++++-- tests/integrations/tso/testutil.go | 3 ++- tests/registry/registry_test.go | 12 +++++++----- tests/scheduling_cluster.go | 1 + tests/server/api/api_test.go | 8 +++++--- tests/server/api/checker_test.go | 1 + tests/server/api/operator_test.go | 4 +++- tests/server/api/region_test.go | 6 ++++-- tests/server/api/rule_test.go | 4 +++- tests/server/api/scheduler_test.go | 6 ++++-- tests/server/api/testutil.go | 1 + tests/server/apiv2/handlers/keyspace_test.go | 8 +++++--- tests/server/apiv2/handlers/testutil.go | 4 +++- .../apiv2/handlers/tso_keyspace_group_test.go | 1 + tests/server/cluster/cluster_test.go | 8 +++++--- tests/server/cluster/cluster_work_test.go | 4 +++- tests/server/config/config_test.go | 1 + tests/server/id/id_test.go | 6 ++++-- tests/server/join/join_fail/join_fail_test.go | 4 +++- tests/server/join/join_test.go | 1 + tests/server/keyspace/keyspace_test.go | 4 +++- tests/server/member/member_test.go | 6 ++++-- .../region_syncer/region_syncer_test.go | 6 ++++-- tests/server/server_test.go | 6 ++++-- .../server/storage/hot_region_storage_test.go | 4 +++- tests/server/tso/tso_test.go | 6 ++++-- tests/server/watch/leader_watch_test.go | 6 ++++-- tests/testutil.go | 6 ++++-- tests/tso_cluster.go | 4 +++- .../pd-analysis/analysis/transfer_counter.go | 3 ++- tools/pd-analysis/main.go | 4 +++- tools/pd-api-bench/cases/cases.go | 6 ++++-- tools/pd-api-bench/cases/controller.go | 6 ++++-- tools/pd-api-bench/config/config.go | 6 ++++-- tools/pd-api-bench/main.go | 14 ++++++++------ tools/pd-backup/main.go | 3 ++- tools/pd-backup/pdbackup/backup.go | 3 ++- tools/pd-backup/pdbackup/backup_test.go | 7 ++++--- tools/pd-backup/tests/backup_test.go | 3 ++- tools/pd-ctl/main.go | 4 +++- tools/pd-ctl/pdctl/command/config_command.go | 1 + tools/pd-ctl/pdctl/command/global.go | 6 ++++-- tools/pd-ctl/pdctl/command/hot_command.go | 4 +++- .../pd-ctl/pdctl/command/keyspace_command.go | 1 + .../pdctl/command/keyspace_group_command.go | 1 + tools/pd-ctl/pdctl/command/label_command.go | 1 + tools/pd-ctl/pdctl/command/log_command.go | 1 + tools/pd-ctl/pdctl/command/operator.go | 3 ++- tools/pd-ctl/pdctl/command/plugin_command.go | 1 + tools/pd-ctl/pdctl/command/region_command.go | 3 ++- tools/pd-ctl/pdctl/command/scheduler.go | 3 ++- tools/pd-ctl/pdctl/command/store_command.go | 6 ++++-- tools/pd-ctl/pdctl/command/tso_command.go | 1 + tools/pd-ctl/pdctl/ctl.go | 1 + tools/pd-ctl/tests/cluster/cluster_test.go | 4 +++- .../tests/completion/completion_test.go | 1 + tools/pd-ctl/tests/config/config_test.go | 6 ++++-- tools/pd-ctl/tests/global_test.go | 4 +++- tools/pd-ctl/tests/health/health_test.go | 3 ++- tools/pd-ctl/tests/helper.go | 1 + tools/pd-ctl/tests/hot/hot_test.go | 6 ++++-- .../tests/keyspace/keyspace_group_test.go | 4 +++- tools/pd-ctl/tests/keyspace/keyspace_test.go | 6 ++++-- tools/pd-ctl/tests/label/label_test.go | 4 +++- tools/pd-ctl/tests/log/log_test.go | 4 +++- tools/pd-ctl/tests/member/member_test.go | 4 +++- tools/pd-ctl/tests/operator/operator_test.go | 4 +++- tools/pd-ctl/tests/region/region_test.go | 4 +++- .../resource_manager_command_test.go | 1 + .../pd-ctl/tests/safepoint/safepoint_test.go | 1 + .../pd-ctl/tests/scheduler/scheduler_test.go | 6 ++++-- tools/pd-ctl/tests/store/store_test.go | 6 ++++-- tools/pd-ctl/tests/tso/tso_test.go | 1 + .../tests/unsafe/unsafe_operation_test.go | 1 + tools/pd-heartbeat-bench/config/config.go | 6 ++++-- tools/pd-heartbeat-bench/main.go | 8 +++++--- tools/pd-heartbeat-bench/metrics/util.go | 3 ++- tools/pd-recover/main.go | 6 ++++-- tools/pd-simulator/main.go | 6 ++++-- .../simulator/cases/balance_leader.go | 2 ++ .../simulator/cases/balance_region.go | 1 + tools/pd-simulator/simulator/cases/cases.go | 1 + .../cases/diagnose_label_isolation.go | 4 +++- .../simulator/cases/diagnose_rule.go | 4 +++- .../pd-simulator/simulator/cases/hot_read.go | 2 ++ .../pd-simulator/simulator/cases/hot_write.go | 2 ++ .../simulator/cases/import_data.go | 4 +++- .../simulator/cases/makeup_down_replica.go | 2 ++ .../simulator/cases/region_merge.go | 2 ++ .../simulator/cases/region_split.go | 2 ++ .../simulator/cases/scale_tikv.go | 1 + .../simulator/cases/stable_env.go | 2 ++ tools/pd-simulator/simulator/client.go | 8 +++++--- tools/pd-simulator/simulator/config/config.go | 1 + tools/pd-simulator/simulator/conn.go | 1 + tools/pd-simulator/simulator/drive.go | 8 +++++--- tools/pd-simulator/simulator/event.go | 4 +++- tools/pd-simulator/simulator/node.go | 4 +++- tools/pd-simulator/simulator/raft.go | 4 +++- tools/pd-simulator/simulator/simutil/key.go | 1 + .../simulator/simutil/key_test.go | 4 +++- .../pd-simulator/simulator/simutil/logger.go | 3 ++- tools/pd-simulator/simulator/task.go | 4 +++- tools/pd-tso-bench/main.go | 10 ++++++---- tools/pd-ut/alloc/check_env_linux.go | 3 ++- tools/pd-ut/alloc/server.go | 4 +++- tools/pd-ut/alloc/tempurl.go | 1 + tools/pd-ut/ut.go | 3 ++- tools/regions-dump/main.go | 6 ++++-- tools/stores-dump/main.go | 6 ++++-- 565 files changed, 1634 insertions(+), 719 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index e13feb04ba5..a44052b22e8 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -28,6 +28,7 @@ linters: - protogetter - reassign - intrange + - gci linters-settings: gocritic: # Which checks should be disabled; can't be combined with 'enabled-checks'; default is empty @@ -233,6 +234,13 @@ linters-settings: desc: "Use 'sync/atomic' instead of 'go.uber.org/atomic'" - pkg: github.com/pkg/errors desc: "Use 'github.com/pingcap/errors' instead of 'github.com/pkg/errors'" + gci: + sections: + - standard + - default + - prefix(github.com/pingcap) + - prefix(github.com/tikv/pd) + - blank issues: exclude-rules: - path: (_test\.go|pkg/mock/.*\.go|tests/.*\.go) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/circuitbreaker/circuit_breaker.go index b5a4c53ebb5..26d1b642ea4 100644 --- a/client/circuitbreaker/circuit_breaker.go +++ b/client/circuitbreaker/circuit_breaker.go @@ -19,13 +19,13 @@ import ( "sync" "time" - "github.com/tikv/pd/client/errs" - "github.com/prometheus/client_golang/prometheus" - m "github.com/tikv/pd/client/metrics" "go.uber.org/zap" "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" + m "github.com/tikv/pd/client/metrics" ) // Overloading is a type describing service return value diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/circuitbreaker/circuit_breaker_test.go index ca77b7f9f99..6a726028cd8 100644 --- a/client/circuitbreaker/circuit_breaker_test.go +++ b/client/circuitbreaker/circuit_breaker_test.go @@ -18,9 +18,9 @@ import ( "testing" "time" - "github.com/tikv/pd/client/errs" - "github.com/stretchr/testify/require" + + "github.com/tikv/pd/client/errs" ) // advance emulate the state machine clock moves forward by the given duration diff --git a/client/client.go b/client/client.go index 519fd478bb3..31bc72c0a77 100644 --- a/client/client.go +++ b/client/client.go @@ -22,15 +22,17 @@ import ( "sync" "time" - cb "github.com/tikv/pd/client/circuitbreaker" - "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/clients/tso" @@ -41,7 +43,6 @@ import ( "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) // GlobalConfigItem standard format of KV pair in GlobalConfig client diff --git a/client/client_test.go b/client/client_test.go index f4f914900cd..305d054fa18 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -20,11 +20,12 @@ import ( "time" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/pkg/utils/testutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/client/clients/metastorage/client.go b/client/clients/metastorage/client.go index dba1127f9f5..baac1ab7539 100644 --- a/client/clients/metastorage/client.go +++ b/client/clients/metastorage/client.go @@ -18,6 +18,7 @@ import ( "context" "github.com/pingcap/kvproto/pkg/meta_storagepb" + "github.com/tikv/pd/client/opt" ) diff --git a/client/clients/router/client.go b/client/clients/router/client.go index 667c82a6805..48cebfa950e 100644 --- a/client/clients/router/client.go +++ b/client/clients/router/client.go @@ -20,6 +20,7 @@ import ( "net/url" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/client/opt" ) diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index 68e2163d191..c26dd25f2ad 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -22,9 +22,16 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" @@ -32,11 +39,6 @@ import ( "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" ) const ( diff --git a/client/clients/tso/dispatcher.go b/client/clients/tso/dispatcher.go index 1b805395904..bdac8096f85 100644 --- a/client/clients/tso/dispatcher.go +++ b/client/clients/tso/dispatcher.go @@ -25,9 +25,12 @@ import ( "time" "github.com/opentracing/opentracing-go" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/client/pkg/utils/timerutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) // deadline is used to control the TS request timeout manually, diff --git a/client/clients/tso/dispatcher_test.go b/client/clients/tso/dispatcher_test.go index 2b5fd1e52e8..cefc53f3944 100644 --- a/client/clients/tso/dispatcher_test.go +++ b/client/clients/tso/dispatcher_test.go @@ -22,13 +22,15 @@ import ( "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" + + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + "github.com/tikv/pd/client/opt" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap/zapcore" ) type mockTSOServiceProvider struct { diff --git a/client/clients/tso/request.go b/client/clients/tso/request.go index 0c9f54f8b2b..9c0d8e6bea6 100644 --- a/client/clients/tso/request.go +++ b/client/clients/tso/request.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/tikv/pd/client/metrics" ) diff --git a/client/clients/tso/request_test.go b/client/clients/tso/request_test.go index 6887ee28124..4be50eaea68 100644 --- a/client/clients/tso/request_test.go +++ b/client/clients/tso/request_test.go @@ -18,8 +18,9 @@ import ( "context" "testing" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" + + "github.com/pingcap/errors" ) func TestTsoRequestWait(t *testing.T) { diff --git a/client/clients/tso/stream.go b/client/clients/tso/stream.go index 6baf63c8882..291d1d31b65 100644 --- a/client/clients/tso/stream.go +++ b/client/clients/tso/stream.go @@ -23,17 +23,19 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" - "go.uber.org/zap" - "google.golang.org/grpc" ) // TSO Stream Builder Factory diff --git a/client/clients/tso/stream_test.go b/client/clients/tso/stream_test.go index 0244c06e024..4791b90bb81 100644 --- a/client/clients/tso/stream_test.go +++ b/client/clients/tso/stream_test.go @@ -21,12 +21,14 @@ import ( "testing" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/client/errs" "go.uber.org/zap/zapcore" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" ) const mockStreamURL = "mock:///" diff --git a/client/errs/errs.go b/client/errs/errs.go index 67a5dd8ec92..868faa6a77a 100644 --- a/client/errs/errs.go +++ b/client/errs/errs.go @@ -17,10 +17,11 @@ package errs import ( "strings" - "github.com/pingcap/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" "google.golang.org/grpc/codes" + + "github.com/pingcap/errors" ) // IsLeaderChange will determine whether there is a leader/primary change. diff --git a/client/gc_client.go b/client/gc_client.go index f30521905c3..45fc8b40b91 100644 --- a/client/gc_client.go +++ b/client/gc_client.go @@ -19,11 +19,13 @@ import ( "time" "github.com/opentracing/opentracing-go" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" - "go.uber.org/zap" ) // GCClient is a client for doing GC diff --git a/client/http/client.go b/client/http/client.go index c813474fcf6..fa9801cf764 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -23,13 +23,15 @@ import ( "net/http" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/retry" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" ) const ( diff --git a/client/http/client_test.go b/client/http/client_test.go index a14691eed02..b6c2105bd4a 100644 --- a/client/http/client_test.go +++ b/client/http/client_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/http/interface.go b/client/http/interface.go index 772599e27fb..1aabd1ae331 100644 --- a/client/http/interface.go +++ b/client/http/interface.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/http/request_info.go b/client/http/request_info.go index 1e3449b59a0..d1930800304 100644 --- a/client/http/request_info.go +++ b/client/http/request_info.go @@ -17,8 +17,9 @@ package http import ( "fmt" - "github.com/tikv/pd/client/pkg/retry" "go.uber.org/zap" + + "github.com/tikv/pd/client/pkg/retry" ) // The following constants are the names of the requests. diff --git a/client/http/types.go b/client/http/types.go index cab564e99ac..83e8badf334 100644 --- a/client/http/types.go +++ b/client/http/types.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" + pd "github.com/tikv/pd/client/clients/router" ) diff --git a/client/inner_client.go b/client/inner_client.go index ae15c763854..045a9a6eed8 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -6,20 +6,21 @@ import ( "sync" "time" + "go.uber.org/zap" + "google.golang.org/grpc" "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" sd "github.com/tikv/pd/client/servicediscovery" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/status" ) const ( diff --git a/client/keyspace_client.go b/client/keyspace_client.go index ce0cc0bc426..84bc29054eb 100644 --- a/client/keyspace_client.go +++ b/client/keyspace_client.go @@ -19,8 +19,10 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" ) diff --git a/client/meta_storage_client.go b/client/meta_storage_client.go index 45bcb8d3d65..fbabd60debd 100644 --- a/client/meta_storage_client.go +++ b/client/meta_storage_client.go @@ -19,10 +19,12 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/prometheus/client_golang/prometheus" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" diff --git a/client/opt/option.go b/client/opt/option.go index 9a80a895cc0..c7a0bb17195 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -18,12 +18,13 @@ import ( "sync/atomic" "time" - cb "github.com/tikv/pd/client/circuitbreaker" + "github.com/prometheus/client_golang/prometheus" + "google.golang.org/grpc" "github.com/pingcap/errors" - "github.com/prometheus/client_golang/prometheus" + + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" - "google.golang.org/grpc" ) const ( diff --git a/client/opt/option_test.go b/client/opt/option_test.go index 26760fac7f2..fa0decbb3a1 100644 --- a/client/opt/option_test.go +++ b/client/opt/option_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/client/pkg/utils/testutil" ) diff --git a/client/pkg/retry/backoff.go b/client/pkg/retry/backoff.go index b0524e6c139..e3b331582d6 100644 --- a/client/pkg/retry/backoff.go +++ b/client/pkg/retry/backoff.go @@ -21,10 +21,11 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "go.uber.org/zap" ) // Option is used to customize the backoffer. diff --git a/client/pkg/retry/backoff_test.go b/client/pkg/retry/backoff_test.go index 12891d822aa..b01a753b374 100644 --- a/client/pkg/retry/backoff_test.go +++ b/client/pkg/retry/backoff_test.go @@ -22,9 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "go.uber.org/zap" + + "github.com/pingcap/log" ) func TestBackoffer(t *testing.T) { diff --git a/client/pkg/utils/grpcutil/grpcutil.go b/client/pkg/utils/grpcutil/grpcutil.go index 29967263dd3..b73d117fe84 100644 --- a/client/pkg/utils/grpcutil/grpcutil.go +++ b/client/pkg/utils/grpcutil/grpcutil.go @@ -21,17 +21,19 @@ import ( "sync" "time" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/tikv/pd/client/errs" - "github.com/tikv/pd/client/pkg/retry" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/backoff" "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/pkg/retry" ) const ( diff --git a/client/pkg/utils/testutil/check_env_linux.go b/client/pkg/utils/testutil/check_env_linux.go index df45f359eb3..ebe6a8e0663 100644 --- a/client/pkg/utils/testutil/check_env_linux.go +++ b/client/pkg/utils/testutil/check_env_linux.go @@ -18,8 +18,9 @@ package testutil import ( "github.com/cakturk/go-netstat/netstat" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) func environmentCheck(addr string) bool { diff --git a/client/pkg/utils/testutil/tempurl.go b/client/pkg/utils/testutil/tempurl.go index 71d51176106..d948160ba42 100644 --- a/client/pkg/utils/testutil/tempurl.go +++ b/client/pkg/utils/testutil/tempurl.go @@ -20,8 +20,9 @@ import ( "sync" "time" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) var ( diff --git a/client/pkg/utils/tlsutil/tlsconfig.go b/client/pkg/utils/tlsutil/tlsconfig.go index 88d797d3b3a..65e0ce9542f 100644 --- a/client/pkg/utils/tlsutil/tlsconfig.go +++ b/client/pkg/utils/tlsutil/tlsconfig.go @@ -40,6 +40,7 @@ import ( "fmt" "github.com/pingcap/errors" + "github.com/tikv/pd/client/errs" ) diff --git a/client/pkg/utils/tlsutil/url.go b/client/pkg/utils/tlsutil/url.go index ccc312d195b..abe4132a6a9 100644 --- a/client/pkg/utils/tlsutil/url.go +++ b/client/pkg/utils/tlsutil/url.go @@ -19,8 +19,9 @@ import ( "net/url" "strings" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) const ( diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index 46401aad1ff..2265053fdd2 100644 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -24,18 +24,20 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "golang.org/x/exp/slices" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" - "go.uber.org/zap" - "golang.org/x/exp/slices" ) const ( diff --git a/client/resource_group/controller/controller_test.go b/client/resource_group/controller/controller_test.go index 254df36020e..f0bdc62d6d3 100644 --- a/client/resource_group/controller/controller_test.go +++ b/client/resource_group/controller/controller_test.go @@ -24,11 +24,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index cd05adb7dfa..c8843da00bd 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -25,10 +25,12 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" - "github.com/tikv/pd/client/errs" "go.uber.org/zap" + + "github.com/pingcap/log" + + "github.com/tikv/pd/client/errs" ) // Limit defines the maximum frequency of some events. diff --git a/client/resource_group/controller/model_test.go b/client/resource_group/controller/model_test.go index 594091da364..99cac79c25a 100644 --- a/client/resource_group/controller/model_test.go +++ b/client/resource_group/controller/model_test.go @@ -17,8 +17,9 @@ package controller import ( "testing" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestGetRUValueFromConsumption(t *testing.T) { diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index 3cf2970109f..0c481631b93 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -19,14 +19,16 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/meta_storagepb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" - "go.uber.org/zap" ) type actionType int diff --git a/client/servicediscovery/pd_service_discovery.go b/client/servicediscovery/pd_service_discovery.go index bd4e442030a..619d4196408 100644 --- a/client/servicediscovery/pd_service_discovery.go +++ b/client/servicediscovery/pd_service_discovery.go @@ -25,21 +25,23 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/retry" "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" ) const ( diff --git a/client/servicediscovery/pd_service_discovery_test.go b/client/servicediscovery/pd_service_discovery_test.go index e8e7ef14e44..dc0a0bd4511 100644 --- a/client/servicediscovery/pd_service_discovery_test.go +++ b/client/servicediscovery/pd_service_discovery_test.go @@ -25,21 +25,23 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/client/errs" - "github.com/tikv/pd/client/opt" - "github.com/tikv/pd/client/pkg/utils/grpcutil" - "github.com/tikv/pd/client/pkg/utils/testutil" - "github.com/tikv/pd/client/pkg/utils/tlsutil" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" pb "google.golang.org/grpc/examples/helloworld/helloworld" "google.golang.org/grpc/health" healthpb "google.golang.org/grpc/health/grpc_health_v1" "google.golang.org/grpc/metadata" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/pdpb" + + "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/opt" + "github.com/tikv/pd/client/pkg/utils/grpcutil" + "github.com/tikv/pd/client/pkg/utils/testutil" + "github.com/tikv/pd/client/pkg/utils/tlsutil" ) type testGRPCServer struct { diff --git a/client/servicediscovery/tso_service_discovery.go b/client/servicediscovery/tso_service_discovery.go index 3fd27837f95..1d2130db804 100644 --- a/client/servicediscovery/tso_service_discovery.go +++ b/client/servicediscovery/tso_service_discovery.go @@ -25,17 +25,19 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/constants" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/utils/grpcutil" - "go.uber.org/zap" - "google.golang.org/grpc" ) const ( diff --git a/cmd/pd-server/main.go b/cmd/pd-server/main.go index 0181d4c47aa..165bcd2a12f 100644 --- a/cmd/pd-server/main.go +++ b/cmd/pd-server/main.go @@ -22,8 +22,11 @@ import ( "syscall" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" - "github.com/pingcap/log" "github.com/spf13/cobra" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/autoscaling" "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/pkg/errs" @@ -41,7 +44,6 @@ import ( "github.com/tikv/pd/server/apiv2" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/join" - "go.uber.org/zap" // register microservice API _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/install" diff --git a/pkg/audit/audit.go b/pkg/audit/audit.go index f84d035f8c9..eba9722f44f 100644 --- a/pkg/audit/audit.go +++ b/pkg/audit/audit.go @@ -17,10 +17,12 @@ package audit import ( "net/http" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" - "github.com/tikv/pd/pkg/utils/requestutil" "go.uber.org/zap" + + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/utils/requestutil" ) const ( diff --git a/pkg/audit/audit_test.go b/pkg/audit/audit_test.go index 9066d81ebe3..c210d91f8b2 100644 --- a/pkg/audit/audit_test.go +++ b/pkg/audit/audit_test.go @@ -27,6 +27,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/requestutil" "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/autoscaling/calculation.go b/pkg/autoscaling/calculation.go index 43aa2972ed8..41db5f6cf78 100644 --- a/pkg/autoscaling/calculation.go +++ b/pkg/autoscaling/calculation.go @@ -20,17 +20,19 @@ import ( "strings" "time" + promClient "github.com/prometheus/client_golang/api" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - promClient "github.com/prometheus/client_golang/api" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/filter" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/autoscaling/calculation_test.go b/pkg/autoscaling/calculation_test.go index 9eb4ad648df..c2a70f8920e 100644 --- a/pkg/autoscaling/calculation_test.go +++ b/pkg/autoscaling/calculation_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/autoscaling/handler.go b/pkg/autoscaling/handler.go index ea248fdcc55..52b342f4a85 100644 --- a/pkg/autoscaling/handler.go +++ b/pkg/autoscaling/handler.go @@ -19,9 +19,10 @@ import ( "io" "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // HTTPHandler is a handler to handle the auto scaling HTTP request. diff --git a/pkg/autoscaling/prometheus.go b/pkg/autoscaling/prometheus.go index 91b813b6ef2..43c47c8c898 100644 --- a/pkg/autoscaling/prometheus.go +++ b/pkg/autoscaling/prometheus.go @@ -22,13 +22,15 @@ import ( "strings" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" promClient "github.com/prometheus/client_golang/api" promAPI "github.com/prometheus/client_golang/api/prometheus/v1" promModel "github.com/prometheus/common/model" - "github.com/tikv/pd/pkg/errs" "go.uber.org/zap" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) const ( diff --git a/pkg/autoscaling/service.go b/pkg/autoscaling/service.go index a8d84b2e598..304f1aa1188 100644 --- a/pkg/autoscaling/service.go +++ b/pkg/autoscaling/service.go @@ -18,11 +18,12 @@ import ( "context" "net/http" + "github.com/unrolled/render" + "github.com/urfave/negroni" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/serverapi" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "github.com/urfave/negroni" ) const autoScalingPrefix = "/autoscaling" diff --git a/pkg/autoscaling/types.go b/pkg/autoscaling/types.go index 53c614312b2..bee0d157191 100644 --- a/pkg/autoscaling/types.go +++ b/pkg/autoscaling/types.go @@ -20,8 +20,9 @@ import ( "regexp" "strings" - "github.com/tikv/pd/pkg/utils/etcdutil" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/pkg/utils/etcdutil" ) // Strategy within a HTTP request provides rules and resources to help make decision for auto scaling. diff --git a/pkg/cache/cache_test.go b/pkg/cache/cache_test.go index 75f26cfed33..d7d7365a344 100644 --- a/pkg/cache/cache_test.go +++ b/pkg/cache/cache_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/cache/ttl.go b/pkg/cache/ttl.go index 2aa39f6c6fd..ba7e3b67330 100644 --- a/pkg/cache/ttl.go +++ b/pkg/cache/ttl.go @@ -18,10 +18,12 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type ttlCacheItem struct { diff --git a/pkg/cgroup/cgmon.go b/pkg/cgroup/cgmon.go index 407e50f50c7..4be033facb1 100644 --- a/pkg/cgroup/cgmon.go +++ b/pkg/cgroup/cgmon.go @@ -21,10 +21,12 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/shirou/gopsutil/v3/mem" - bs "github.com/tikv/pd/pkg/basicserver" "go.uber.org/zap" + + "github.com/pingcap/log" + + bs "github.com/tikv/pd/pkg/basicserver" ) const ( diff --git a/pkg/cgroup/cgroup.go b/pkg/cgroup/cgroup.go index 133bd3158c8..0093020a0ac 100644 --- a/pkg/cgroup/cgroup.go +++ b/pkg/cgroup/cgroup.go @@ -26,9 +26,10 @@ import ( "strconv" "strings" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "go.uber.org/zap" ) // CPUQuotaStatus presents the status of how CPU quota is used diff --git a/pkg/core/factory_test.go b/pkg/core/factory_test.go index 9f0ba883885..a81da9af081 100644 --- a/pkg/core/factory_test.go +++ b/pkg/core/factory_test.go @@ -18,7 +18,9 @@ import ( "testing" "github.com/gogo/protobuf/proto" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/core/region.go b/pkg/core/region.go index b5ead86b3f1..5f5a4a5f2e0 100644 --- a/pkg/core/region.go +++ b/pkg/core/region.go @@ -29,17 +29,19 @@ import ( "github.com/docker/go-units" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/core/region_test.go b/pkg/core/region_test.go index 8c30efb2769..51ba5fe96dc 100644 --- a/pkg/core/region_test.go +++ b/pkg/core/region_test.go @@ -23,10 +23,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mock/mockid" diff --git a/pkg/core/region_tree.go b/pkg/core/region_tree.go index 12e2c5c8878..6efafd133cf 100644 --- a/pkg/core/region_tree.go +++ b/pkg/core/region_tree.go @@ -18,12 +18,14 @@ import ( "bytes" "math/rand" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/btree" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) type regionItem struct { diff --git a/pkg/core/region_tree_test.go b/pkg/core/region_tree_test.go index 0dedd91be9e..4d18394aa70 100644 --- a/pkg/core/region_tree_test.go +++ b/pkg/core/region_tree_test.go @@ -19,9 +19,10 @@ import ( "math/rand" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestRegionInfo(t *testing.T) { diff --git a/pkg/core/store.go b/pkg/core/store.go index 5edf59f6dce..9ec9a44bfc8 100644 --- a/pkg/core/store.go +++ b/pkg/core/store.go @@ -20,15 +20,17 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/core/store_option.go b/pkg/core/store_option.go index 3d05a0fb6e1..80fb09853e2 100644 --- a/pkg/core/store_option.go +++ b/pkg/core/store_option.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/core/store_stats.go b/pkg/core/store_stats.go index d68f8b8e43c..6c6a6420d56 100644 --- a/pkg/core/store_stats.go +++ b/pkg/core/store_stats.go @@ -16,6 +16,7 @@ package core import ( "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/core/store_stats_test.go b/pkg/core/store_stats_test.go index 9f2969cd81f..47f30dbd27b 100644 --- a/pkg/core/store_stats_test.go +++ b/pkg/core/store_stats_test.go @@ -18,9 +18,10 @@ import ( "testing" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestStoreStats(t *testing.T) { diff --git a/pkg/core/store_test.go b/pkg/core/store_test.go index 5cb324e5635..7c5aaa98289 100644 --- a/pkg/core/store_test.go +++ b/pkg/core/store_test.go @@ -21,9 +21,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/core/storelimit/limit_test.go b/pkg/core/storelimit/limit_test.go index cfe805935a5..eb0bde2732f 100644 --- a/pkg/core/storelimit/limit_test.go +++ b/pkg/core/storelimit/limit_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/dashboard/adapter/manager.go b/pkg/dashboard/adapter/manager.go index 293d8ad6549..648f5562419 100644 --- a/pkg/dashboard/adapter/manager.go +++ b/pkg/dashboard/adapter/manager.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/pingcap/tidb-dashboard/pkg/apiserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/server" diff --git a/pkg/dashboard/adapter/redirector.go b/pkg/dashboard/adapter/redirector.go index c1d845065b7..5d0fd1801cd 100644 --- a/pkg/dashboard/adapter/redirector.go +++ b/pkg/dashboard/adapter/redirector.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb-dashboard/pkg/apiserver" "github.com/pingcap/tidb-dashboard/pkg/utils" + "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/dashboard/distroutil/distro.go b/pkg/dashboard/distroutil/distro.go index a19db806d70..0bc7b1d031b 100644 --- a/pkg/dashboard/distroutil/distro.go +++ b/pkg/dashboard/distroutil/distro.go @@ -18,9 +18,10 @@ import ( "os" "path/filepath" + "go.uber.org/zap" + "github.com/pingcap/log" "github.com/pingcap/tidb-dashboard/util/distro" - "go.uber.org/zap" ) const ( diff --git a/pkg/dashboard/keyvisual/input/core.go b/pkg/dashboard/keyvisual/input/core.go index 3ca5f96cd81..0219d6e0b77 100644 --- a/pkg/dashboard/keyvisual/input/core.go +++ b/pkg/dashboard/keyvisual/input/core.go @@ -15,11 +15,13 @@ package input import ( + "go.uber.org/zap" + "github.com/pingcap/log" regionpkg "github.com/pingcap/tidb-dashboard/pkg/keyvisual/region" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/server" - "go.uber.org/zap" ) const limit = 1024 diff --git a/pkg/dashboard/uiserver/embedded_assets_rewriter.go b/pkg/dashboard/uiserver/embedded_assets_rewriter.go index d19db01936f..f0f9f0c1e51 100644 --- a/pkg/dashboard/uiserver/embedded_assets_rewriter.go +++ b/pkg/dashboard/uiserver/embedded_assets_rewriter.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb-dashboard/pkg/config" "github.com/pingcap/tidb-dashboard/pkg/uiserver" + "github.com/tikv/pd/pkg/dashboard/distroutil" ) diff --git a/pkg/election/leadership.go b/pkg/election/leadership.go index ec64a003c53..bb0ce7bf361 100644 --- a/pkg/election/leadership.go +++ b/pkg/election/leadership.go @@ -19,18 +19,20 @@ import ( "sync/atomic" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/election/leadership_test.go b/pkg/election/leadership_test.go index eab842ca6e5..fc158e6f73a 100644 --- a/pkg/election/leadership_test.go +++ b/pkg/election/leadership_test.go @@ -21,12 +21,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/pingcap/failpoint" + + "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/testutil" ) const defaultLeaseTimeout = 1 diff --git a/pkg/election/lease.go b/pkg/election/lease.go index 21bd43018b5..1dff5ea7a99 100644 --- a/pkg/election/lease.go +++ b/pkg/election/lease.go @@ -19,13 +19,15 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/election/lease_test.go b/pkg/election/lease_test.go index 3a02de97239..b099d0c0d5e 100644 --- a/pkg/election/lease_test.go +++ b/pkg/election/lease_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/encryption/config.go b/pkg/encryption/config.go index d4d257a8a43..38222c56b5d 100644 --- a/pkg/encryption/config.go +++ b/pkg/encryption/config.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/encryption/config_test.go b/pkg/encryption/config_test.go index 4134d46c2f3..2ab5ae68dfa 100644 --- a/pkg/encryption/config_test.go +++ b/pkg/encryption/config_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/encryption/crypter.go b/pkg/encryption/crypter.go index 7e69854c5a8..5c91228c150 100644 --- a/pkg/encryption/crypter.go +++ b/pkg/encryption/crypter.go @@ -22,6 +22,7 @@ import ( "io" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/crypter_test.go b/pkg/encryption/crypter_test.go index 9ac72bd7813..ff660cc1a9d 100644 --- a/pkg/encryption/crypter_test.go +++ b/pkg/encryption/crypter_test.go @@ -19,8 +19,9 @@ import ( "encoding/hex" "testing" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/encryptionpb" ) func TestEncryptionMethodSupported(t *testing.T) { diff --git a/pkg/encryption/key_manager.go b/pkg/encryption/key_manager.go index 621b1b9742f..54e5fa01b35 100644 --- a/pkg/encryption/key_manager.go +++ b/pkg/encryption/key_manager.go @@ -20,15 +20,17 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/encryption/key_manager_test.go b/pkg/encryption/key_manager_test.go index f387497c2e9..52e91296314 100644 --- a/pkg/encryption/key_manager_test.go +++ b/pkg/encryption/key_manager_test.go @@ -24,13 +24,15 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // #nosec G101 diff --git a/pkg/encryption/kms.go b/pkg/encryption/kms.go index 99dcf9619a3..3836cf53db6 100644 --- a/pkg/encryption/kms.go +++ b/pkg/encryption/kms.go @@ -22,7 +22,9 @@ import ( "github.com/aws/aws-sdk-go-v2/credentials/stscreds" "github.com/aws/aws-sdk-go-v2/service/kms" "github.com/aws/aws-sdk-go-v2/service/sts" + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/master_key.go b/pkg/encryption/master_key.go index 31d9a0cb591..aff480386ac 100644 --- a/pkg/encryption/master_key.go +++ b/pkg/encryption/master_key.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/encryption/master_key_test.go b/pkg/encryption/master_key_test.go index d6d7845284a..0a8e99bf75a 100644 --- a/pkg/encryption/master_key_test.go +++ b/pkg/encryption/master_key_test.go @@ -20,8 +20,9 @@ import ( "path/filepath" "testing" - "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/encryptionpb" ) func TestPlaintextMasterKey(t *testing.T) { diff --git a/pkg/encryption/region_crypter.go b/pkg/encryption/region_crypter.go index 458c5b67d7b..96826d8bef6 100644 --- a/pkg/encryption/region_crypter.go +++ b/pkg/encryption/region_crypter.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/encryption/region_crypter_test.go b/pkg/encryption/region_crypter_test.go index b1ca558063c..1b6f910b0d5 100644 --- a/pkg/encryption/region_crypter_test.go +++ b/pkg/encryption/region_crypter_test.go @@ -19,10 +19,11 @@ import ( "crypto/cipher" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" ) type testKeyManager struct { diff --git a/pkg/errs/errs.go b/pkg/errs/errs.go index 5746b282f10..84cc90312d1 100644 --- a/pkg/errs/errs.go +++ b/pkg/errs/errs.go @@ -15,9 +15,10 @@ package errs import ( - "github.com/pingcap/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + + "github.com/pingcap/errors" ) // ZapError is used to make the log output easier. diff --git a/pkg/errs/errs_test.go b/pkg/errs/errs_test.go index 01b7de461b8..80722f4a8c4 100644 --- a/pkg/errs/errs_test.go +++ b/pkg/errs/errs_test.go @@ -20,10 +20,11 @@ import ( "strings" "testing" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "go.uber.org/zap" + + "github.com/pingcap/errors" + "github.com/pingcap/log" ) // testingWriter is a WriteSyncer that writes to the the messages. diff --git a/pkg/gc/safepoint.go b/pkg/gc/safepoint.go index c1b4687e109..5bc64ae9a90 100644 --- a/pkg/gc/safepoint.go +++ b/pkg/gc/safepoint.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/config" diff --git a/pkg/gc/safepoint_test.go b/pkg/gc/safepoint_test.go index 62ce9c086fc..b67f7b4fe44 100644 --- a/pkg/gc/safepoint_test.go +++ b/pkg/gc/safepoint_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/server/config" diff --git a/pkg/gc/safepoint_v2.go b/pkg/gc/safepoint_v2.go index 665249bcab0..449be8f3d59 100644 --- a/pkg/gc/safepoint_v2.go +++ b/pkg/gc/safepoint_v2.go @@ -18,16 +18,18 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) var ( diff --git a/pkg/gctuner/memory_limit_tuner.go b/pkg/gctuner/memory_limit_tuner.go index 8a852b191d8..10e28842938 100644 --- a/pkg/gctuner/memory_limit_tuner.go +++ b/pkg/gctuner/memory_limit_tuner.go @@ -20,12 +20,14 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + util "github.com/tikv/pd/pkg/gogc" "github.com/tikv/pd/pkg/memory" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // GlobalMemoryLimitTuner only allow one memory limit tuner in one process diff --git a/pkg/gctuner/memory_limit_tuner_test.go b/pkg/gctuner/memory_limit_tuner_test.go index 5e5f84ccbac..6c0aaca685d 100644 --- a/pkg/gctuner/memory_limit_tuner_test.go +++ b/pkg/gctuner/memory_limit_tuner_test.go @@ -20,8 +20,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/memory" ) diff --git a/pkg/gctuner/tuner.go b/pkg/gctuner/tuner.go index 74932fe174b..bc63b36c5a2 100644 --- a/pkg/gctuner/tuner.go +++ b/pkg/gctuner/tuner.go @@ -20,9 +20,11 @@ import ( "strconv" "sync/atomic" + "go.uber.org/zap" + "github.com/pingcap/log" + util "github.com/tikv/pd/pkg/gogc" - "go.uber.org/zap" ) var ( diff --git a/pkg/id/id.go b/pkg/id/id.go index eb2788fc656..cb38c23268d 100644 --- a/pkg/id/id.go +++ b/pkg/id/id.go @@ -15,16 +15,18 @@ package id import ( - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) type label string diff --git a/pkg/id/id_test.go b/pkg/id/id_test.go index e08c0e93367..9cd8493c430 100644 --- a/pkg/id/id_test.go +++ b/pkg/id/id_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 08bb51da936..4a50f36169f 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -20,10 +20,13 @@ import ( "strconv" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/keyspace/keyspace_test.go b/pkg/keyspace/keyspace_test.go index 3c259649cd3..57aeb70341d 100644 --- a/pkg/keyspace/keyspace_test.go +++ b/pkg/keyspace/keyspace_test.go @@ -23,10 +23,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index fa26b4cd0cb..dc414cfad3a 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -23,10 +23,15 @@ import ( "sync" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/balancer" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -38,9 +43,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 5878d7d907f..c615627be71 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 91d07676205..95b3162eed0 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/keyspace/util_test.go b/pkg/keyspace/util_test.go index ab544b21a5d..f938904c709 100644 --- a/pkg/keyspace/util_test.go +++ b/pkg/keyspace/util_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/mcs/discovery/discover.go b/pkg/mcs/discovery/discover.go index d4234df893d..f46c72495c6 100644 --- a/pkg/mcs/discovery/discover.go +++ b/pkg/mcs/discovery/discover.go @@ -15,15 +15,17 @@ package discovery import ( + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "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/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Discover is used to get all the service instances of the specified service name. diff --git a/pkg/mcs/discovery/discover_test.go b/pkg/mcs/discovery/discover_test.go index fd66ddcad18..81af0d524b7 100644 --- a/pkg/mcs/discovery/discover_test.go +++ b/pkg/mcs/discovery/discover_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" ) diff --git a/pkg/mcs/discovery/register.go b/pkg/mcs/discovery/register.go index 5ab0ceabfce..e62239a4694 100644 --- a/pkg/mcs/discovery/register.go +++ b/pkg/mcs/discovery/register.go @@ -19,12 +19,14 @@ import ( "fmt" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // DefaultLeaseInSeconds is the default lease time in seconds. diff --git a/pkg/mcs/discovery/register_test.go b/pkg/mcs/discovery/register_test.go index bdaf7e379a4..b03543f624e 100644 --- a/pkg/mcs/discovery/register_test.go +++ b/pkg/mcs/discovery/register_test.go @@ -22,10 +22,11 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" - "github.com/tikv/pd/pkg/utils/testutil" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/tikv/pd/pkg/utils/etcdutil" + "github.com/tikv/pd/pkg/utils/testutil" ) func TestRegister(t *testing.T) { diff --git a/pkg/mcs/discovery/registry_entry.go b/pkg/mcs/discovery/registry_entry.go index db4ac44a2cc..887a8eb7aea 100644 --- a/pkg/mcs/discovery/registry_entry.go +++ b/pkg/mcs/discovery/registry_entry.go @@ -17,8 +17,9 @@ package discovery import ( "encoding/json" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) // ServiceRegistryEntry is the registry entry of a service diff --git a/pkg/mcs/metastorage/server/grpc_service.go b/pkg/mcs/metastorage/server/grpc_service.go index 32b3788906d..00f4efb56fd 100644 --- a/pkg/mcs/metastorage/server/grpc_service.go +++ b/pkg/mcs/metastorage/server/grpc_service.go @@ -19,17 +19,19 @@ import ( "fmt" "net/http" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var ( diff --git a/pkg/mcs/metastorage/server/manager.go b/pkg/mcs/metastorage/server/manager.go index 49fc58c6b7d..0c03e4c7d03 100644 --- a/pkg/mcs/metastorage/server/manager.go +++ b/pkg/mcs/metastorage/server/manager.go @@ -15,12 +15,14 @@ package server import ( + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Manager is the manager of resource group. diff --git a/pkg/mcs/registry/registry.go b/pkg/mcs/registry/registry.go index c6470b34dc4..6a01f091e52 100644 --- a/pkg/mcs/registry/registry.go +++ b/pkg/mcs/registry/registry.go @@ -20,10 +20,12 @@ import ( "fmt" "net/http" - "github.com/pingcap/log" - bs "github.com/tikv/pd/pkg/basicserver" "go.uber.org/zap" "google.golang.org/grpc" + + "github.com/pingcap/log" + + bs "github.com/tikv/pd/pkg/basicserver" ) var ( diff --git a/pkg/mcs/resourcemanager/server/apis/v1/api.go b/pkg/mcs/resourcemanager/server/apis/v1/api.go index 891072e898f..e142dbbc69a 100644 --- a/pkg/mcs/resourcemanager/server/apis/v1/api.go +++ b/pkg/mcs/resourcemanager/server/apis/v1/api.go @@ -25,7 +25,9 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" + rmserver "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/pkg/mcs/resourcemanager/server/config.go b/pkg/mcs/resourcemanager/server/config.go index 360f3b169b9..415a87d5e79 100644 --- a/pkg/mcs/resourcemanager/server/config.go +++ b/pkg/mcs/resourcemanager/server/config.go @@ -22,16 +22,18 @@ import ( "time" "github.com/BurntSushi/toml" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/grpc_service.go b/pkg/mcs/resourcemanager/server/grpc_service.go index 21681bc0759..6c0d7ce0120 100644 --- a/pkg/mcs/resourcemanager/server/grpc_service.go +++ b/pkg/mcs/resourcemanager/server/grpc_service.go @@ -20,17 +20,19 @@ import ( "net/http" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var ( diff --git a/pkg/mcs/resourcemanager/server/manager.go b/pkg/mcs/resourcemanager/server/manager.go index 7f7e710b3fb..2ccfd44c418 100644 --- a/pkg/mcs/resourcemanager/server/manager.go +++ b/pkg/mcs/resourcemanager/server/manager.go @@ -23,11 +23,14 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/jsonutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/metrics_test.go b/pkg/mcs/resourcemanager/server/metrics_test.go index d69d364b64b..4c3ec7ce5ef 100644 --- a/pkg/mcs/resourcemanager/server/metrics_test.go +++ b/pkg/mcs/resourcemanager/server/metrics_test.go @@ -18,8 +18,9 @@ import ( "fmt" "testing" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestMaxPerSecCostTracker(t *testing.T) { diff --git a/pkg/mcs/resourcemanager/server/resource_group.go b/pkg/mcs/resourcemanager/server/resource_group.go index 6a5d06da6f7..65d2959e870 100644 --- a/pkg/mcs/resourcemanager/server/resource_group.go +++ b/pkg/mcs/resourcemanager/server/resource_group.go @@ -20,12 +20,14 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + "github.com/pingcap/errors" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // ResourceGroup is the definition of a resource group, for REST API. diff --git a/pkg/mcs/resourcemanager/server/resource_group_test.go b/pkg/mcs/resourcemanager/server/resource_group_test.go index 96b15d14293..8f058a212bb 100644 --- a/pkg/mcs/resourcemanager/server/resource_group_test.go +++ b/pkg/mcs/resourcemanager/server/resource_group_test.go @@ -5,8 +5,9 @@ import ( "testing" "github.com/brianvoe/gofakeit/v6" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestPatchResourceGroup(t *testing.T) { diff --git a/pkg/mcs/resourcemanager/server/token_buckets.go b/pkg/mcs/resourcemanager/server/token_buckets.go index f72d0dfcc88..50dc78c9d68 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets.go +++ b/pkg/mcs/resourcemanager/server/token_buckets.go @@ -19,9 +19,10 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.uber.org/zap" + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/resourcemanager/server/token_buckets_test.go b/pkg/mcs/resourcemanager/server/token_buckets_test.go index 3d9cbd3f628..b56ccb6ab96 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets_test.go +++ b/pkg/mcs/resourcemanager/server/token_buckets_test.go @@ -20,8 +20,9 @@ import ( "testing" "time" - rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/stretchr/testify/require" + + rmpb "github.com/pingcap/kvproto/pkg/resource_manager" ) func TestGroupTokenBucketUpdateAndPatch(t *testing.T) { diff --git a/pkg/mcs/scheduling/server/apis/v1/api.go b/pkg/mcs/scheduling/server/apis/v1/api.go index c374456a6eb..3d2d0005a24 100644 --- a/pkg/mcs/scheduling/server/apis/v1/api.go +++ b/pkg/mcs/scheduling/server/apis/v1/api.go @@ -27,8 +27,11 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/unrolled/render" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" scheserver "github.com/tikv/pd/pkg/mcs/scheduling/server" mcsutils "github.com/tikv/pd/pkg/mcs/utils" @@ -44,7 +47,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - "github.com/unrolled/render" ) // APIPathPrefix is the prefix of the API path. diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index e4949446da9..20e5acca379 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -7,12 +7,15 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Cluster is used to manage all information for scheduling purpose. diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index 45a606720a0..784d1f45a82 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -27,10 +27,13 @@ import ( "github.com/BurntSushi/toml" "github.com/coreos/go-semver/semver" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -43,7 +46,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index f0af61c9dd2..f499a0d7d50 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -23,15 +23,17 @@ import ( "time" "github.com/coreos/go-semver/semver" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any configuration changes. diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index f4fe606b403..440b2d47d4f 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -21,10 +21,16 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -33,10 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // gRPC errors diff --git a/pkg/mcs/scheduling/server/meta/watcher.go b/pkg/mcs/scheduling/server/meta/watcher.go index 9d54a636b9e..27fe6687f3d 100644 --- a/pkg/mcs/scheduling/server/meta/watcher.go +++ b/pkg/mcs/scheduling/server/meta/watcher.go @@ -20,15 +20,17 @@ import ( "sync" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any meta changes. diff --git a/pkg/mcs/scheduling/server/rule/watcher.go b/pkg/mcs/scheduling/server/rule/watcher.go index 790dd9c2f81..cc6480a0cb4 100644 --- a/pkg/mcs/scheduling/server/rule/watcher.go +++ b/pkg/mcs/scheduling/server/rule/watcher.go @@ -19,7 +19,12 @@ import ( "strings" "sync" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/checker" "github.com/tikv/pd/pkg/schedule/labeler" @@ -27,9 +32,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Watcher is used to watch the PD API server for any Placement Rule changes. diff --git a/pkg/mcs/scheduling/server/rule/watcher_test.go b/pkg/mcs/scheduling/server/rule/watcher_test.go index 40469eef2a8..5b3d49a53f1 100644 --- a/pkg/mcs/scheduling/server/rule/watcher_test.go +++ b/pkg/mcs/scheduling/server/rule/watcher_test.go @@ -24,14 +24,15 @@ import ( "time" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" ) const ( diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 31c8d307adb..8c9972d5eec 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -28,6 +28,10 @@ import ( "time" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/spf13/cobra" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -35,7 +39,7 @@ import ( "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/spf13/cobra" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" @@ -62,8 +66,6 @@ import ( "github.com/tikv/pd/pkg/utils/memberutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" ) var _ bs.Server = (*Server)(nil) diff --git a/pkg/mcs/scheduling/server/testutil.go b/pkg/mcs/scheduling/server/testutil.go index 312365c81ab..794a1bf520c 100644 --- a/pkg/mcs/scheduling/server/testutil.go +++ b/pkg/mcs/scheduling/server/testutil.go @@ -18,9 +18,11 @@ import ( "context" "os" - "github.com/pingcap/log" "github.com/spf13/pflag" "github.com/stretchr/testify/require" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/pkg/mcs/server/server.go b/pkg/mcs/server/server.go index fef05a85012..9692b52beb3 100644 --- a/pkg/mcs/server/server.go +++ b/pkg/mcs/server/server.go @@ -23,11 +23,13 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "google.golang.org/grpc" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/grpcutil" - clientv3 "go.etcd.io/etcd/client/v3" - "google.golang.org/grpc" ) // BaseServer is a basic server that provides some common functionality. diff --git a/pkg/mcs/tso/server/apis/v1/api.go b/pkg/mcs/tso/server/apis/v1/api.go index ec8782eb522..d659c13edca 100644 --- a/pkg/mcs/tso/server/apis/v1/api.go +++ b/pkg/mcs/tso/server/apis/v1/api.go @@ -23,8 +23,12 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" tsoserver "github.com/tikv/pd/pkg/mcs/tso/server" "github.com/tikv/pd/pkg/mcs/utils" @@ -34,8 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/multiservicesapi" "github.com/tikv/pd/pkg/utils/logutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/tso/server/config.go b/pkg/mcs/tso/server/config.go index 2aaa54114da..f2ee3c36a7b 100644 --- a/pkg/mcs/tso/server/config.go +++ b/pkg/mcs/tso/server/config.go @@ -22,16 +22,18 @@ import ( "time" "github.com/BurntSushi/toml" + "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/mcs/tso/server/config_test.go b/pkg/mcs/tso/server/config_test.go index 2bafec30aa9..08686bd5208 100644 --- a/pkg/mcs/tso/server/config_test.go +++ b/pkg/mcs/tso/server/config_test.go @@ -21,6 +21,7 @@ import ( "github.com/BurntSushi/toml" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mcs/utils/constant" ) diff --git a/pkg/mcs/tso/server/grpc_service.go b/pkg/mcs/tso/server/grpc_service.go index 3419fd16221..59abed67213 100644 --- a/pkg/mcs/tso/server/grpc_service.go +++ b/pkg/mcs/tso/server/grpc_service.go @@ -21,16 +21,18 @@ import ( "strconv" "time" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // gRPC errors diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index d2974075e94..34f51573baf 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -27,12 +27,18 @@ import ( "time" grpcprometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/spf13/cobra" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/diagnosticspb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/spf13/cobra" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -49,10 +55,6 @@ import ( "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) var _ bs.Server = (*Server)(nil) diff --git a/pkg/mcs/tso/server/testutil.go b/pkg/mcs/tso/server/testutil.go index 5dcfd4759b9..5ffcac48edb 100644 --- a/pkg/mcs/tso/server/testutil.go +++ b/pkg/mcs/tso/server/testutil.go @@ -17,11 +17,12 @@ package server import ( "strings" - "github.com/pingcap/kvproto/pkg/tsopb" "github.com/spf13/pflag" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/kvproto/pkg/tsopb" ) // MustNewGrpcClient must create a new TSO grpc client. diff --git a/pkg/mcs/utils/expected_primary.go b/pkg/mcs/utils/expected_primary.go index c39345b004e..b8a317ed251 100644 --- a/pkg/mcs/utils/expected_primary.go +++ b/pkg/mcs/utils/expected_primary.go @@ -20,8 +20,12 @@ import ( "math/rand" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -29,8 +33,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // GetExpectedPrimaryFlag gets the expected primary flag. diff --git a/pkg/mcs/utils/util.go b/pkg/mcs/utils/util.go index 253c846d167..68e7edb3d69 100644 --- a/pkg/mcs/utils/util.go +++ b/pkg/mcs/utils/util.go @@ -25,11 +25,18 @@ import ( "time" "github.com/gin-gonic/gin" - "github.com/pingcap/kvproto/pkg/diagnosticspb" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + + "github.com/pingcap/kvproto/pkg/diagnosticspb" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -40,11 +47,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/versioninfo" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) // PromHandler is a handler to get prometheus metrics. diff --git a/pkg/member/member.go b/pkg/member/member.go index 04e55c1a647..8b388cdee6a 100644 --- a/pkg/member/member.go +++ b/pkg/member/member.go @@ -24,18 +24,20 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) const ( diff --git a/pkg/member/participant.go b/pkg/member/participant.go index f3399f5d900..5d9129bcad5 100644 --- a/pkg/member/participant.go +++ b/pkg/member/participant.go @@ -19,16 +19,18 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) type leadershipCheckFunc func(*election.Leadership) bool diff --git a/pkg/memory/meminfo.go b/pkg/memory/meminfo.go index 64505e2e5f3..7ed1afb579b 100644 --- a/pkg/memory/meminfo.go +++ b/pkg/memory/meminfo.go @@ -17,14 +17,16 @@ package memory import ( "time" + "github.com/shirou/gopsutil/v3/mem" + "go.uber.org/zap" + "golang.org/x/exp/constraints" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/sysutil" - "github.com/shirou/gopsutil/v3/mem" + "github.com/tikv/pd/pkg/cgroup" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "golang.org/x/exp/constraints" ) // MemTotal returns the total amount of RAM on this system diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index 5fe17e5e723..45d8e35a0bc 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -21,10 +21,12 @@ import ( "time" "github.com/docker/go-units" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/mock/mockhbstream/mockhbstream.go b/pkg/mock/mockhbstream/mockhbstream.go index ac8f246f86a..848f134c8a0 100644 --- a/pkg/mock/mockhbstream/mockhbstream.go +++ b/pkg/mock/mockhbstream/mockhbstream.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/hbstream" ) diff --git a/pkg/mock/mockhbstream/mockhbstream_test.go b/pkg/mock/mockhbstream/mockhbstream_test.go index 87a39b028b9..e4ffe8a9b20 100644 --- a/pkg/mock/mockhbstream/mockhbstream_test.go +++ b/pkg/mock/mockhbstream/mockhbstream_test.go @@ -18,10 +18,12 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/hbstream" diff --git a/pkg/mock/mockserver/mockserver.go b/pkg/mock/mockserver/mockserver.go index d79d79ffa03..3ba2abfca3f 100644 --- a/pkg/mock/mockserver/mockserver.go +++ b/pkg/mock/mockserver/mockserver.go @@ -18,6 +18,7 @@ import ( "context" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/grpcutil" diff --git a/pkg/ratelimit/controller_test.go b/pkg/ratelimit/controller_test.go index 5efa6ec1190..f42169c9902 100644 --- a/pkg/ratelimit/controller_test.go +++ b/pkg/ratelimit/controller_test.go @@ -21,8 +21,9 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) type changeAndResult struct { diff --git a/pkg/ratelimit/limiter.go b/pkg/ratelimit/limiter.go index e312066dc56..0fc3a4e6bd1 100644 --- a/pkg/ratelimit/limiter.go +++ b/pkg/ratelimit/limiter.go @@ -17,9 +17,10 @@ package ratelimit import ( "math" + "golang.org/x/time/rate" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" - "golang.org/x/time/rate" ) // DoneFunc is done function. diff --git a/pkg/ratelimit/limiter_test.go b/pkg/ratelimit/limiter_test.go index 256f9ea9ab4..2bb967677bb 100644 --- a/pkg/ratelimit/limiter_test.go +++ b/pkg/ratelimit/limiter_test.go @@ -20,8 +20,9 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) type releaseUtil struct { diff --git a/pkg/ratelimit/ratelimiter.go b/pkg/ratelimit/ratelimiter.go index b88127915c9..1c2a6a0f7b0 100644 --- a/pkg/ratelimit/ratelimiter.go +++ b/pkg/ratelimit/ratelimiter.go @@ -18,8 +18,9 @@ import ( "context" "time" - "github.com/tikv/pd/pkg/utils/syncutil" "golang.org/x/time/rate" + + "github.com/tikv/pd/pkg/utils/syncutil" ) // RateLimiter is a rate limiter based on `golang.org/x/time/rate`. diff --git a/pkg/ratelimit/runner.go b/pkg/ratelimit/runner.go index 1d65ff6a568..211a4f71be1 100644 --- a/pkg/ratelimit/runner.go +++ b/pkg/ratelimit/runner.go @@ -20,9 +20,10 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + + "github.com/pingcap/log" ) // RegionHeartbeatStageName is the name of the stage of the region heartbeat. diff --git a/pkg/replication/replication_mode.go b/pkg/replication/replication_mode.go index cba83cf1ebb..856a68a9b09 100644 --- a/pkg/replication/replication_mode.go +++ b/pkg/replication/replication_mode.go @@ -24,9 +24,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" pb "github.com/pingcap/kvproto/pkg/replication_modepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/config" - "go.uber.org/zap" ) const ( diff --git a/pkg/replication/replication_mode_test.go b/pkg/replication/replication_mode_test.go index 243d7f7d8f1..0a921bccf3a 100644 --- a/pkg/replication/replication_mode_test.go +++ b/pkg/replication/replication_mode_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/pdpb" pb "github.com/pingcap/kvproto/pkg/replication_modepb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/response/region.go b/pkg/response/region.go index 6db7f135ad8..b635780e4f2 100644 --- a/pkg/response/region.go +++ b/pkg/response/region.go @@ -18,9 +18,11 @@ import ( "context" "github.com/mailru/easyjson/jwriter" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/response/region_test.go b/pkg/response/region_test.go index de6daa2c2fe..8da4c739f79 100644 --- a/pkg/response/region_test.go +++ b/pkg/response/region_test.go @@ -18,9 +18,10 @@ import ( "encoding/json" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" ) func TestPeer(t *testing.T) { diff --git a/pkg/response/store.go b/pkg/response/store.go index 8bff1e75e42..25dc7fc2176 100644 --- a/pkg/response/store.go +++ b/pkg/response/store.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/pkg/schedule/checker/checker_controller.go b/pkg/schedule/checker/checker_controller.go index 49d097daea6..aa15ee03d6f 100644 --- a/pkg/schedule/checker/checker_controller.go +++ b/pkg/schedule/checker/checker_controller.go @@ -22,8 +22,11 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/checker/joint_state_checker.go b/pkg/schedule/checker/joint_state_checker.go index 2122044e64a..4dc3922906a 100644 --- a/pkg/schedule/checker/joint_state_checker.go +++ b/pkg/schedule/checker/joint_state_checker.go @@ -16,6 +16,7 @@ package checker import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/checker/joint_state_checker_test.go b/pkg/schedule/checker/joint_state_checker_test.go index 1b9436b65fd..f649c737284 100644 --- a/pkg/schedule/checker/joint_state_checker_test.go +++ b/pkg/schedule/checker/joint_state_checker_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/checker/learner_checker.go b/pkg/schedule/checker/learner_checker.go index 8590904a760..6d830c39ad8 100644 --- a/pkg/schedule/checker/learner_checker.go +++ b/pkg/schedule/checker/learner_checker.go @@ -16,6 +16,7 @@ package checker import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/checker/learner_checker_test.go b/pkg/schedule/checker/learner_checker_test.go index b5d994d4201..3a559b86958 100644 --- a/pkg/schedule/checker/learner_checker_test.go +++ b/pkg/schedule/checker/learner_checker_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/checker/merge_checker.go b/pkg/schedule/checker/merge_checker.go index bf7fe4f2496..571ee134da0 100644 --- a/pkg/schedule/checker/merge_checker.go +++ b/pkg/schedule/checker/merge_checker.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" diff --git a/pkg/schedule/checker/merge_checker_test.go b/pkg/schedule/checker/merge_checker_test.go index 00aaafa2cfd..3737e6cc0c2 100644 --- a/pkg/schedule/checker/merge_checker_test.go +++ b/pkg/schedule/checker/merge_checker_test.go @@ -20,8 +20,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/utils/operatorutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/pkg/schedule/checker/priority_inspector_test.go b/pkg/schedule/checker/priority_inspector_test.go index 5a5d8079d93..6de76b5ccb1 100644 --- a/pkg/schedule/checker/priority_inspector_test.go +++ b/pkg/schedule/checker/priority_inspector_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" ) diff --git a/pkg/schedule/checker/replica_checker.go b/pkg/schedule/checker/replica_checker.go index fabee683c92..f1d6efe15ba 100644 --- a/pkg/schedule/checker/replica_checker.go +++ b/pkg/schedule/checker/replica_checker.go @@ -19,8 +19,11 @@ import ( "math/rand" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -29,7 +32,6 @@ import ( sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/checker/replica_checker_test.go b/pkg/schedule/checker/replica_checker_test.go index da04fb6d768..3f0f6b67d8d 100644 --- a/pkg/schedule/checker/replica_checker_test.go +++ b/pkg/schedule/checker/replica_checker_test.go @@ -20,10 +20,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/checker/replica_strategy.go b/pkg/schedule/checker/replica_strategy.go index ffb25f90ad5..3bf2c955af7 100644 --- a/pkg/schedule/checker/replica_strategy.go +++ b/pkg/schedule/checker/replica_strategy.go @@ -17,12 +17,14 @@ package checker import ( "math/rand" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" - "go.uber.org/zap" ) // ReplicaStrategy collects some utilities to manipulate region peers. It diff --git a/pkg/schedule/checker/rule_checker.go b/pkg/schedule/checker/rule_checker.go index 2d06f84fdfe..28d5988ce1c 100644 --- a/pkg/schedule/checker/rule_checker.go +++ b/pkg/schedule/checker/rule_checker.go @@ -21,9 +21,12 @@ import ( "math/rand" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) const maxPendingListLen = 100000 diff --git a/pkg/schedule/checker/rule_checker_test.go b/pkg/schedule/checker/rule_checker_test.go index 5ac67122de1..0b37ed57e7e 100644 --- a/pkg/schedule/checker/rule_checker_test.go +++ b/pkg/schedule/checker/rule_checker_test.go @@ -17,16 +17,17 @@ package checker import ( "context" "fmt" - "strconv" "strings" "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/pkg/schedule/checker/split_checker.go b/pkg/schedule/checker/split_checker.go index 3cc1664b6cc..91487949821 100644 --- a/pkg/schedule/checker/split_checker.go +++ b/pkg/schedule/checker/split_checker.go @@ -17,6 +17,7 @@ package checker import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/checker/split_checker_test.go b/pkg/schedule/checker/split_checker_test.go index 40357d2408c..be3cce84f0d 100644 --- a/pkg/schedule/checker/split_checker_test.go +++ b/pkg/schedule/checker/split_checker_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/pkg/schedule/config/config.go b/pkg/schedule/config/config.go index 5e8f2c587ac..f00cd8a943b 100644 --- a/pkg/schedule/config/config.go +++ b/pkg/schedule/config/config.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/configutil" diff --git a/pkg/schedule/config/config_provider.go b/pkg/schedule/config/config_provider.go index 5c1be1089e9..a18051ca38e 100644 --- a/pkg/schedule/config/config_provider.go +++ b/pkg/schedule/config/config_provider.go @@ -19,7 +19,9 @@ import ( "time" "github.com/coreos/go-semver/semver" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/config/store_config.go b/pkg/schedule/config/store_config.go index b7676311f3d..19cca753d1d 100644 --- a/pkg/schedule/config/store_config.go +++ b/pkg/schedule/config/store_config.go @@ -18,10 +18,12 @@ import ( "encoding/json" "reflect" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) var ( diff --git a/pkg/schedule/config/util_test.go b/pkg/schedule/config/util_test.go index 31ab3ccf2a6..bda6b9441ad 100644 --- a/pkg/schedule/config/util_test.go +++ b/pkg/schedule/config/util_test.go @@ -17,8 +17,9 @@ package config import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func TestValidateLabels(t *testing.T) { diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 739ca5c84b6..e792560cb37 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -20,9 +20,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/checker" @@ -39,7 +42,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/filter/candidates_test.go b/pkg/schedule/filter/candidates_test.go index fd03b42ace0..ce8be5ef72e 100644 --- a/pkg/schedule/filter/candidates_test.go +++ b/pkg/schedule/filter/candidates_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/plan" diff --git a/pkg/schedule/filter/counter_test.go b/pkg/schedule/filter/counter_test.go index 7c7acc5e9a5..74ff194c47a 100644 --- a/pkg/schedule/filter/counter_test.go +++ b/pkg/schedule/filter/counter_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/filter/filters.go b/pkg/schedule/filter/filters.go index 4ea59935109..efb27c3ec6d 100644 --- a/pkg/schedule/filter/filters.go +++ b/pkg/schedule/filter/filters.go @@ -17,8 +17,11 @@ package filter import ( "strconv" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) // SelectSourceStores selects stores that be selected as source store from the list. diff --git a/pkg/schedule/filter/filters_test.go b/pkg/schedule/filter/filters_test.go index f061a472d65..35ba2c33589 100644 --- a/pkg/schedule/filter/filters_test.go +++ b/pkg/schedule/filter/filters_test.go @@ -19,9 +19,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/schedule/filter/healthy_test.go b/pkg/schedule/filter/healthy_test.go index 15588352554..dd5ab8e958d 100644 --- a/pkg/schedule/filter/healthy_test.go +++ b/pkg/schedule/filter/healthy_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/filter/region_filters_test.go b/pkg/schedule/filter/region_filters_test.go index a7dd1fa932a..ddd4141f8ba 100644 --- a/pkg/schedule/filter/region_filters_test.go +++ b/pkg/schedule/filter/region_filters_test.go @@ -19,9 +19,11 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/handler/handler.go b/pkg/schedule/handler/handler.go index a8540b4b5f4..042adf96611 100644 --- a/pkg/schedule/handler/handler.go +++ b/pkg/schedule/handler/handler.go @@ -23,11 +23,14 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule" @@ -43,7 +46,6 @@ import ( "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/hbstream/heartbeat_streams.go b/pkg/schedule/hbstream/heartbeat_streams.go index ef0440a9f77..6637800e97f 100644 --- a/pkg/schedule/hbstream/heartbeat_streams.go +++ b/pkg/schedule/hbstream/heartbeat_streams.go @@ -20,17 +20,19 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Operation is detailed scheduling step of a region. diff --git a/pkg/schedule/labeler/labeler.go b/pkg/schedule/labeler/labeler.go index 7670ccdedd7..16a14068e68 100644 --- a/pkg/schedule/labeler/labeler.go +++ b/pkg/schedule/labeler/labeler.go @@ -19,7 +19,10 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/rangelist" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // RegionLabeler is utility to label regions. diff --git a/pkg/schedule/labeler/labeler_test.go b/pkg/schedule/labeler/labeler_test.go index ebd57708e47..564542727ff 100644 --- a/pkg/schedule/labeler/labeler_test.go +++ b/pkg/schedule/labeler/labeler_test.go @@ -24,8 +24,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/schedule/labeler/rules.go b/pkg/schedule/labeler/rules.go index 39a420032d8..4f5d28a8034 100644 --- a/pkg/schedule/labeler/rules.go +++ b/pkg/schedule/labeler/rules.go @@ -23,10 +23,12 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // RegionLabel is the label of a region. diff --git a/pkg/schedule/operator/builder.go b/pkg/schedule/operator/builder.go index 29b8aedf978..8fd2393da2e 100644 --- a/pkg/schedule/operator/builder.go +++ b/pkg/schedule/operator/builder.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" diff --git a/pkg/schedule/operator/builder_test.go b/pkg/schedule/operator/builder_test.go index 5e9ff7f89bb..77224a82c15 100644 --- a/pkg/schedule/operator/builder_test.go +++ b/pkg/schedule/operator/builder_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/create_operator.go b/pkg/schedule/operator/create_operator.go index 4fae7f9e3f2..2558ca8d75b 100644 --- a/pkg/schedule/operator/create_operator.go +++ b/pkg/schedule/operator/create_operator.go @@ -18,16 +18,18 @@ import ( "fmt" "math/rand" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // CreateAddPeerOperator creates an operator that adds a new peer. diff --git a/pkg/schedule/operator/create_operator_test.go b/pkg/schedule/operator/create_operator_test.go index 845255e713c..0143b1b8011 100644 --- a/pkg/schedule/operator/create_operator_test.go +++ b/pkg/schedule/operator/create_operator_test.go @@ -19,11 +19,13 @@ import ( "encoding/hex" "testing" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/metrics.go b/pkg/schedule/operator/metrics.go index 47a165500e9..25de41f0fb7 100644 --- a/pkg/schedule/operator/metrics.go +++ b/pkg/schedule/operator/metrics.go @@ -16,6 +16,7 @@ package operator import ( "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/operator/operator.go b/pkg/schedule/operator/operator.go index 8abeae54f6b..69b2504755e 100644 --- a/pkg/schedule/operator/operator.go +++ b/pkg/schedule/operator/operator.go @@ -22,8 +22,10 @@ import ( "sync/atomic" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/schedule/operator/operator_controller.go b/pkg/schedule/operator/operator_controller.go index 0478ef2b6ae..cd2470376e1 100644 --- a/pkg/schedule/operator/operator_controller.go +++ b/pkg/schedule/operator/operator_controller.go @@ -21,9 +21,12 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) // The source of dispatched region. diff --git a/pkg/schedule/operator/operator_controller_test.go b/pkg/schedule/operator/operator_controller_test.go index 69600f80536..3c9abe54f24 100644 --- a/pkg/schedule/operator/operator_controller_test.go +++ b/pkg/schedule/operator/operator_controller_test.go @@ -22,11 +22,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/operator/operator_test.go b/pkg/schedule/operator/operator_test.go index 22a86c789fc..6976b5ca12e 100644 --- a/pkg/schedule/operator/operator_test.go +++ b/pkg/schedule/operator/operator_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/schedule/operator/step.go b/pkg/schedule/operator/step.go index 04e41028865..6b0f0f1021f 100644 --- a/pkg/schedule/operator/step.go +++ b/pkg/schedule/operator/step.go @@ -20,17 +20,19 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/operator/step_test.go b/pkg/schedule/operator/step_test.go index 014703d00f9..f6323a6d23c 100644 --- a/pkg/schedule/operator/step_test.go +++ b/pkg/schedule/operator/step_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/operator/test_util.go b/pkg/schedule/operator/test_util.go index 8206189daa6..f86701792dd 100644 --- a/pkg/schedule/operator/test_util.go +++ b/pkg/schedule/operator/test_util.go @@ -16,6 +16,7 @@ package operator import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" ) diff --git a/pkg/schedule/operator/waiting_operator_test.go b/pkg/schedule/operator/waiting_operator_test.go index 8a0d1875cd7..a2e333231b0 100644 --- a/pkg/schedule/operator/waiting_operator_test.go +++ b/pkg/schedule/operator/waiting_operator_test.go @@ -17,8 +17,10 @@ package operator import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core/constant" ) diff --git a/pkg/schedule/placement/fit.go b/pkg/schedule/placement/fit.go index 30530462664..bb1d642900b 100644 --- a/pkg/schedule/placement/fit.go +++ b/pkg/schedule/placement/fit.go @@ -20,6 +20,7 @@ import ( "sort" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/fit_region_test.go b/pkg/schedule/placement/fit_region_test.go index 284674b79a2..9c6e7b7faf1 100644 --- a/pkg/schedule/placement/fit_region_test.go +++ b/pkg/schedule/placement/fit_region_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/fit_test.go b/pkg/schedule/placement/fit_test.go index b12bcd7451a..f22661ba694 100644 --- a/pkg/schedule/placement/fit_test.go +++ b/pkg/schedule/placement/fit_test.go @@ -21,9 +21,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/label_constraint_test.go b/pkg/schedule/placement/label_constraint_test.go index e65676f0a5e..cd2ff5976df 100644 --- a/pkg/schedule/placement/label_constraint_test.go +++ b/pkg/schedule/placement/label_constraint_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/region_rule_cache.go b/pkg/schedule/placement/region_rule_cache.go index 79e52a49fca..4b17c3baf70 100644 --- a/pkg/schedule/placement/region_rule_cache.go +++ b/pkg/schedule/placement/region_rule_cache.go @@ -16,6 +16,7 @@ package placement import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/syncutil" diff --git a/pkg/schedule/placement/region_rule_cache_test.go b/pkg/schedule/placement/region_rule_cache_test.go index e951ea10cc5..c526a095e7b 100644 --- a/pkg/schedule/placement/region_rule_cache_test.go +++ b/pkg/schedule/placement/region_rule_cache_test.go @@ -19,9 +19,11 @@ import ( "time" "unsafe" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/placement/rule_list.go b/pkg/schedule/placement/rule_list.go index f5ee0dada0e..3fc401163d1 100644 --- a/pkg/schedule/placement/rule_list.go +++ b/pkg/schedule/placement/rule_list.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/rangelist" ) diff --git a/pkg/schedule/placement/rule_manager.go b/pkg/schedule/placement/rule_manager.go index f44258d797c..f5101f0250c 100644 --- a/pkg/schedule/placement/rule_manager.go +++ b/pkg/schedule/placement/rule_manager.go @@ -24,8 +24,12 @@ import ( "sort" "strings" + "go.uber.org/zap" + "golang.org/x/exp/slices" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -35,8 +39,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "golang.org/x/exp/slices" ) const ( diff --git a/pkg/schedule/placement/rule_manager_test.go b/pkg/schedule/placement/rule_manager_test.go index 2e1883640d8..5f5f457da13 100644 --- a/pkg/schedule/placement/rule_manager_test.go +++ b/pkg/schedule/placement/rule_manager_test.go @@ -19,8 +19,10 @@ import ( "encoding/hex" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/pkg/schedule/plan/balance_plan_test.go b/pkg/schedule/plan/balance_plan_test.go index 59f2acc689a..8bdc2519a90 100644 --- a/pkg/schedule/plan/balance_plan_test.go +++ b/pkg/schedule/plan/balance_plan_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/plugin_interface.go b/pkg/schedule/plugin_interface.go index 62ffe2eb900..8870ca7f46d 100644 --- a/pkg/schedule/plugin_interface.go +++ b/pkg/schedule/plugin_interface.go @@ -18,10 +18,12 @@ import ( "path/filepath" "plugin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // PluginInterface is used to manage all plugin. diff --git a/pkg/schedule/prepare_checker.go b/pkg/schedule/prepare_checker.go index 187d68e3d9f..5f6b54f33ae 100644 --- a/pkg/schedule/prepare_checker.go +++ b/pkg/schedule/prepare_checker.go @@ -17,10 +17,12 @@ package schedule import ( "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type prepareChecker struct { diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index 71b2cd346c7..fdcbd705398 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -22,10 +22,13 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" @@ -36,7 +39,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const regionScatterName = "region-scatter" diff --git a/pkg/schedule/scatter/region_scatterer_test.go b/pkg/schedule/scatter/region_scatterer_test.go index fdaed888309..b86d73d4288 100644 --- a/pkg/schedule/scatter/region_scatterer_test.go +++ b/pkg/schedule/scatter/region_scatterer_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_benchmark_test.go b/pkg/schedule/schedulers/balance_benchmark_test.go index aaafa7be8ca..abf4c0b3def 100644 --- a/pkg/schedule/schedulers/balance_benchmark_test.go +++ b/pkg/schedule/schedulers/balance_benchmark_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/assert" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/balance_leader.go b/pkg/schedule/schedulers/balance_leader.go index 03f02002c74..2884e8a486f 100644 --- a/pkg/schedule/schedulers/balance_leader.go +++ b/pkg/schedule/schedulers/balance_leader.go @@ -23,7 +23,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -34,8 +38,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/typeutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/balance_leader_test.go b/pkg/schedule/schedulers/balance_leader_test.go index 940f75f78bc..36a4ee52ea6 100644 --- a/pkg/schedule/schedulers/balance_leader_test.go +++ b/pkg/schedule/schedulers/balance_leader_test.go @@ -22,9 +22,11 @@ import ( "testing" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_region.go b/pkg/schedule/schedulers/balance_region.go index 2bee521364d..6ad95201f68 100644 --- a/pkg/schedule/schedulers/balance_region.go +++ b/pkg/schedule/schedulers/balance_region.go @@ -18,8 +18,11 @@ import ( "sort" "strconv" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -27,7 +30,6 @@ import ( "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type balanceRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/balance_region_test.go b/pkg/schedule/schedulers/balance_region_test.go index 48a4959a170..aa8e7fc30d1 100644 --- a/pkg/schedule/schedulers/balance_region_test.go +++ b/pkg/schedule/schedulers/balance_region_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/schedule/schedulers/balance_witness.go b/pkg/schedule/schedulers/balance_witness.go index cfcafb56c57..1b23e5b4a19 100644 --- a/pkg/schedule/schedulers/balance_witness.go +++ b/pkg/schedule/schedulers/balance_witness.go @@ -23,8 +23,12 @@ import ( "strconv" "github.com/gorilla/mux" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + "github.com/unrolled/render" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -35,8 +39,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/balance_witness_test.go b/pkg/schedule/schedulers/balance_witness_test.go index b1449821236..a7edbd3ea94 100644 --- a/pkg/schedule/schedulers/balance_witness_test.go +++ b/pkg/schedule/schedulers/balance_witness_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/base_scheduler.go b/pkg/schedule/schedulers/base_scheduler.go index ef5c6bf7ae7..2534ed12281 100644 --- a/pkg/schedule/schedulers/base_scheduler.go +++ b/pkg/schedule/schedulers/base_scheduler.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/config.go b/pkg/schedule/schedulers/config.go index 78b123981fd..e172b596685 100644 --- a/pkg/schedule/schedulers/config.go +++ b/pkg/schedule/schedulers/config.go @@ -15,13 +15,15 @@ package schedulers import ( + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type schedulerConfig interface { diff --git a/pkg/schedule/schedulers/config_test.go b/pkg/schedule/schedulers/config_test.go index 9e20521854f..f97a2ad17ea 100644 --- a/pkg/schedule/schedulers/config_test.go +++ b/pkg/schedule/schedulers/config_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage" ) diff --git a/pkg/schedule/schedulers/evict_leader.go b/pkg/schedule/schedulers/evict_leader.go index defc65846ae..45285b51137 100644 --- a/pkg/schedule/schedulers/evict_leader.go +++ b/pkg/schedule/schedulers/evict_leader.go @@ -20,8 +20,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -32,8 +36,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_leader_test.go b/pkg/schedule/schedulers/evict_leader_test.go index ecd80813c0f..587a48358e9 100644 --- a/pkg/schedule/schedulers/evict_leader_test.go +++ b/pkg/schedule/schedulers/evict_leader_test.go @@ -18,9 +18,11 @@ import ( "bytes" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/schedulers/evict_slow_store.go b/pkg/schedule/schedulers/evict_slow_store.go index b1960ceca8e..8d8c014b110 100644 --- a/pkg/schedule/schedulers/evict_slow_store.go +++ b/pkg/schedule/schedulers/evict_slow_store.go @@ -19,9 +19,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -29,8 +33,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_slow_store_test.go b/pkg/schedule/schedulers/evict_slow_store_test.go index 79651fb5b5c..636d856fc14 100644 --- a/pkg/schedule/schedulers/evict_slow_store_test.go +++ b/pkg/schedule/schedulers/evict_slow_store_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/evict_slow_trend.go b/pkg/schedule/schedulers/evict_slow_trend.go index 92805587f72..cf4bf3d3b39 100644 --- a/pkg/schedule/schedulers/evict_slow_trend.go +++ b/pkg/schedule/schedulers/evict_slow_trend.go @@ -20,9 +20,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/evict_slow_trend_test.go b/pkg/schedule/schedulers/evict_slow_trend_test.go index 4be6eeb58d9..21420cc6022 100644 --- a/pkg/schedule/schedulers/evict_slow_trend_test.go +++ b/pkg/schedule/schedulers/evict_slow_trend_test.go @@ -19,9 +19,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 88b9f5c6c93..837d7290ff7 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -21,8 +21,12 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -36,8 +40,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type grantHotRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/grant_leader.go b/pkg/schedule/schedulers/grant_leader.go index b18d1fc3397..d0cb6bae34d 100644 --- a/pkg/schedule/schedulers/grant_leader.go +++ b/pkg/schedule/schedulers/grant_leader.go @@ -19,8 +19,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type grantLeaderSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/grant_leader_test.go b/pkg/schedule/schedulers/grant_leader_test.go index 38bac3c961a..003966db73d 100644 --- a/pkg/schedule/schedulers/grant_leader_test.go +++ b/pkg/schedule/schedulers/grant_leader_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/hot_region.go b/pkg/schedule/schedulers/hot_region.go index 97a558c3fe4..622e617b729 100644 --- a/pkg/schedule/schedulers/hot_region.go +++ b/pkg/schedule/schedulers/hot_region.go @@ -23,10 +23,13 @@ import ( "strconv" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -41,7 +44,6 @@ import ( "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/hot_region_config.go b/pkg/schedule/schedulers/hot_region_config.go index df82ccc3afc..c458d29842f 100644 --- a/pkg/schedule/schedulers/hot_region_config.go +++ b/pkg/schedule/schedulers/hot_region_config.go @@ -22,7 +22,11 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/slice" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/hot_region_rank_v2_test.go b/pkg/schedule/schedulers/hot_region_rank_v2_test.go index f00e9dde787..d4cbfc092a0 100644 --- a/pkg/schedule/schedulers/hot_region_rank_v2_test.go +++ b/pkg/schedule/schedulers/hot_region_rank_v2_test.go @@ -19,6 +19,7 @@ import ( "github.com/docker/go-units" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index 9f79ac617c9..b37fc00cea1 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -22,9 +22,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/schedule/operator" diff --git a/pkg/schedule/schedulers/label.go b/pkg/schedule/schedulers/label.go index a27ea29687e..cd63b6c0511 100644 --- a/pkg/schedule/schedulers/label.go +++ b/pkg/schedule/schedulers/label.go @@ -15,7 +15,10 @@ package schedulers import ( + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -25,7 +28,6 @@ import ( "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type labelSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/metrics.go b/pkg/schedule/schedulers/metrics.go index 4afc4605f52..bd8a2b4f6ea 100644 --- a/pkg/schedule/schedulers/metrics.go +++ b/pkg/schedule/schedulers/metrics.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/schedule/types" ) diff --git a/pkg/schedule/schedulers/random_merge.go b/pkg/schedule/schedulers/random_merge.go index 7cd9954ce4b..ccb1bc12f9a 100644 --- a/pkg/schedule/schedulers/random_merge.go +++ b/pkg/schedule/schedulers/random_merge.go @@ -18,6 +18,7 @@ import ( "math/rand" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/random_merge_test.go b/pkg/schedule/schedulers/random_merge_test.go index 6c8ea353f2d..e7be6d589ba 100644 --- a/pkg/schedule/schedulers/random_merge_test.go +++ b/pkg/schedule/schedulers/random_merge_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/range_cluster.go b/pkg/schedule/schedulers/range_cluster.go index e83e74145f6..0b6e908b56c 100644 --- a/pkg/schedule/schedulers/range_cluster.go +++ b/pkg/schedule/schedulers/range_cluster.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/docker/go-units" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" ) diff --git a/pkg/schedule/schedulers/scatter_range.go b/pkg/schedule/schedulers/scatter_range.go index 5ba303ad05a..2e071ec6ca7 100644 --- a/pkg/schedule/schedulers/scatter_range.go +++ b/pkg/schedule/schedulers/scatter_range.go @@ -19,7 +19,10 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -28,7 +31,6 @@ import ( "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) type scatterRangeSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/scatter_range_test.go b/pkg/schedule/schedulers/scatter_range_test.go index 26ac48a36bb..7125ba9a596 100644 --- a/pkg/schedule/schedulers/scatter_range_test.go +++ b/pkg/schedule/schedulers/scatter_range_test.go @@ -19,8 +19,10 @@ import ( "math/rand" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage" diff --git a/pkg/schedule/schedulers/scheduler.go b/pkg/schedule/schedulers/scheduler.go index 3f722f7f804..8976c3a1928 100644 --- a/pkg/schedule/schedulers/scheduler.go +++ b/pkg/schedule/schedulers/scheduler.go @@ -20,8 +20,11 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/config" sche "github.com/tikv/pd/pkg/schedule/core" @@ -29,7 +32,6 @@ import ( "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/storage/endpoint" - "go.uber.org/zap" ) // Scheduler is an interface to schedule resources. diff --git a/pkg/schedule/schedulers/scheduler_controller.go b/pkg/schedule/schedulers/scheduler_controller.go index cb4ffd6f9c2..28973631570 100644 --- a/pkg/schedule/schedulers/scheduler_controller.go +++ b/pkg/schedule/schedulers/scheduler_controller.go @@ -22,7 +22,10 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const maxScheduleRetries = 10 diff --git a/pkg/schedule/schedulers/scheduler_test.go b/pkg/schedule/schedulers/scheduler_test.go index 0e06b9333e9..24df62ad3e1 100644 --- a/pkg/schedule/schedulers/scheduler_test.go +++ b/pkg/schedule/schedulers/scheduler_test.go @@ -20,8 +20,10 @@ import ( "testing" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/schedule/schedulers/shuffle_hot_region.go b/pkg/schedule/schedulers/shuffle_hot_region.go index 7517abb3c21..6da4b71ea45 100644 --- a/pkg/schedule/schedulers/shuffle_hot_region.go +++ b/pkg/schedule/schedulers/shuffle_hot_region.go @@ -18,8 +18,12 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" @@ -30,8 +34,6 @@ import ( "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" - "go.uber.org/zap" ) type shuffleHotRegionSchedulerConfig struct { diff --git a/pkg/schedule/schedulers/shuffle_leader.go b/pkg/schedule/schedulers/shuffle_leader.go index e2a256af7a7..8fd5b87c1bc 100644 --- a/pkg/schedule/schedulers/shuffle_leader.go +++ b/pkg/schedule/schedulers/shuffle_leader.go @@ -16,6 +16,7 @@ package schedulers import ( "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/shuffle_region.go b/pkg/schedule/schedulers/shuffle_region.go index 1fbc1f08e67..2b4e2443210 100644 --- a/pkg/schedule/schedulers/shuffle_region.go +++ b/pkg/schedule/schedulers/shuffle_region.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" sche "github.com/tikv/pd/pkg/schedule/core" diff --git a/pkg/schedule/schedulers/shuffle_region_config.go b/pkg/schedule/schedulers/shuffle_region_config.go index 2e3394a58df..58ea25b6186 100644 --- a/pkg/schedule/schedulers/shuffle_region_config.go +++ b/pkg/schedule/schedulers/shuffle_region_config.go @@ -18,12 +18,13 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/pkg/schedule/schedulers/split_bucket.go b/pkg/schedule/schedulers/split_bucket.go index edbe2ac3545..d6aee65b181 100644 --- a/pkg/schedule/schedulers/split_bucket.go +++ b/pkg/schedule/schedulers/split_bucket.go @@ -22,8 +22,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/operator" @@ -32,7 +35,6 @@ import ( "github.com/tikv/pd/pkg/statistics/buckets" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/pkg/schedule/schedulers/split_bucket_test.go b/pkg/schedule/schedulers/split_bucket_test.go index 840dfa97c19..41892bb4fc4 100644 --- a/pkg/schedule/schedulers/split_bucket_test.go +++ b/pkg/schedule/schedulers/split_bucket_test.go @@ -18,8 +18,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/statistics/buckets" diff --git a/pkg/schedule/schedulers/transfer_witness_leader.go b/pkg/schedule/schedulers/transfer_witness_leader.go index 90191dd355c..c84f0918884 100644 --- a/pkg/schedule/schedulers/transfer_witness_leader.go +++ b/pkg/schedule/schedulers/transfer_witness_leader.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/schedule/schedulers/transfer_witness_leader_test.go b/pkg/schedule/schedulers/transfer_witness_leader_test.go index b100e0a9535..e58f3dd3544 100644 --- a/pkg/schedule/schedulers/transfer_witness_leader_test.go +++ b/pkg/schedule/schedulers/transfer_witness_leader_test.go @@ -17,9 +17,11 @@ package schedulers import ( "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/types" diff --git a/pkg/schedule/schedulers/utils.go b/pkg/schedule/schedulers/utils.go index 9c1c7fe3854..9a6a6bd96fa 100644 --- a/pkg/schedule/schedulers/utils.go +++ b/pkg/schedule/schedulers/utils.go @@ -19,7 +19,10 @@ import ( "strconv" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -28,7 +31,6 @@ import ( "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/schedule/plan" "github.com/tikv/pd/pkg/statistics" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/schedulers/utils_test.go b/pkg/schedule/schedulers/utils_test.go index deb7c6e1038..d85fba47ff4 100644 --- a/pkg/schedule/schedulers/utils_test.go +++ b/pkg/schedule/schedulers/utils_test.go @@ -17,8 +17,10 @@ package schedulers import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/schedule/splitter/region_splitter.go b/pkg/schedule/splitter/region_splitter.go index 37b33dad480..c5b950d73be 100644 --- a/pkg/schedule/splitter/region_splitter.go +++ b/pkg/schedule/splitter/region_splitter.go @@ -22,15 +22,17 @@ import ( "math" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/schedule/filter" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/schedule/splitter/region_splitter_test.go b/pkg/schedule/splitter/region_splitter_test.go index 6f49707217e..8e59fa74382 100644 --- a/pkg/schedule/splitter/region_splitter_test.go +++ b/pkg/schedule/splitter/region_splitter_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/slice/slice_test.go b/pkg/slice/slice_test.go index 019cd49c46a..c0abb46f298 100644 --- a/pkg/slice/slice_test.go +++ b/pkg/slice/slice_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/slice" ) diff --git a/pkg/statistics/buckets/hot_bucket_cache.go b/pkg/statistics/buckets/hot_bucket_cache.go index 3fc640e8c1f..f29ef1563e8 100644 --- a/pkg/statistics/buckets/hot_bucket_cache.go +++ b/pkg/statistics/buckets/hot_bucket_cache.go @@ -19,12 +19,14 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/rangetree" "github.com/tikv/pd/pkg/utils/keyutil" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) type status int diff --git a/pkg/statistics/buckets/hot_bucket_cache_test.go b/pkg/statistics/buckets/hot_bucket_cache_test.go index e9669eb2eec..06e00687fd7 100644 --- a/pkg/statistics/buckets/hot_bucket_cache_test.go +++ b/pkg/statistics/buckets/hot_bucket_cache_test.go @@ -18,8 +18,9 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func TestPutItem(t *testing.T) { diff --git a/pkg/statistics/buckets/hot_bucket_task_test.go b/pkg/statistics/buckets/hot_bucket_task_test.go index 100ed7c818d..93551919ba2 100644 --- a/pkg/statistics/buckets/hot_bucket_task_test.go +++ b/pkg/statistics/buckets/hot_bucket_task_test.go @@ -21,8 +21,9 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" ) func getAllBucketStats(ctx context.Context, hotCache *HotBucketCache) map[uint64][]*BucketStat { diff --git a/pkg/statistics/hot_cache.go b/pkg/statistics/hot_cache.go index ae61063646d..99ec587aea7 100644 --- a/pkg/statistics/hot_cache.go +++ b/pkg/statistics/hot_cache.go @@ -17,8 +17,10 @@ package statistics import ( "context" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/smallnest/chanx" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/logutil" diff --git a/pkg/statistics/hot_cache_test.go b/pkg/statistics/hot_cache_test.go index b0232b658d4..668c930adcc 100644 --- a/pkg/statistics/hot_cache_test.go +++ b/pkg/statistics/hot_cache_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/statistics/hot_peer.go b/pkg/statistics/hot_peer.go index c17ad5c246f..55dcd03b4ee 100644 --- a/pkg/statistics/hot_peer.go +++ b/pkg/statistics/hot_peer.go @@ -18,12 +18,14 @@ import ( "math" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) type dimStat struct { diff --git a/pkg/statistics/hot_peer_cache.go b/pkg/statistics/hot_peer_cache.go index b62248062bd..77b5d567ca4 100644 --- a/pkg/statistics/hot_peer_cache.go +++ b/pkg/statistics/hot_peer_cache.go @@ -20,9 +20,11 @@ import ( "math" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" "github.com/smallnest/chanx" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/pkg/statistics/hot_peer_cache_test.go b/pkg/statistics/hot_peer_cache_test.go index 2d44b5dc783..09d267d60b9 100644 --- a/pkg/statistics/hot_peer_cache_test.go +++ b/pkg/statistics/hot_peer_cache_test.go @@ -23,9 +23,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/movingaverage" diff --git a/pkg/statistics/region_collection.go b/pkg/statistics/region_collection.go index 958ba3be5df..8d12dbf42c9 100644 --- a/pkg/statistics/region_collection.go +++ b/pkg/statistics/region_collection.go @@ -17,12 +17,14 @@ package statistics import ( "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // RegionInfoProvider is an interface to provide the region information. diff --git a/pkg/statistics/region_collection_test.go b/pkg/statistics/region_collection_test.go index 64a625a04e2..f97dffa893d 100644 --- a/pkg/statistics/region_collection_test.go +++ b/pkg/statistics/region_collection_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockconfig" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/pkg/statistics/store.go b/pkg/statistics/store.go index baeef0ad417..fbbfea72700 100644 --- a/pkg/statistics/store.go +++ b/pkg/statistics/store.go @@ -17,13 +17,15 @@ package statistics import ( "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/movingaverage" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/statistics/store_collection.go b/pkg/statistics/store_collection.go index f55c23b27b7..a7c77aa7b4b 100644 --- a/pkg/statistics/store_collection.go +++ b/pkg/statistics/store_collection.go @@ -19,6 +19,7 @@ import ( "strconv" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" diff --git a/pkg/statistics/store_collection_test.go b/pkg/statistics/store_collection_test.go index 6a0ef24aff5..93845d4de1a 100644 --- a/pkg/statistics/store_collection_test.go +++ b/pkg/statistics/store_collection_test.go @@ -20,9 +20,11 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/mock/mockconfig" diff --git a/pkg/statistics/store_load_test.go b/pkg/statistics/store_load_test.go index 9d958151182..cbcbb0bacfe 100644 --- a/pkg/statistics/store_load_test.go +++ b/pkg/statistics/store_load_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/statistics/store_test.go b/pkg/statistics/store_test.go index ccf85caaa72..741fdc54fd3 100644 --- a/pkg/statistics/store_test.go +++ b/pkg/statistics/store_test.go @@ -18,9 +18,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/statistics/utils/kind_test.go b/pkg/statistics/utils/kind_test.go index 0a02ffa00c1..d0834e077da 100644 --- a/pkg/statistics/utils/kind_test.go +++ b/pkg/statistics/utils/kind_test.go @@ -17,9 +17,11 @@ package utils import ( "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" ) diff --git a/pkg/storage/endpoint/cluster_id.go b/pkg/storage/endpoint/cluster_id.go index 974b65d6c6c..f9317d44874 100644 --- a/pkg/storage/endpoint/cluster_id.go +++ b/pkg/storage/endpoint/cluster_id.go @@ -19,14 +19,16 @@ import ( "math/rand" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // InitClusterID creates a cluster ID if it hasn't existed. diff --git a/pkg/storage/endpoint/cluster_id_test.go b/pkg/storage/endpoint/cluster_id_test.go index 5ce1600044d..dcd06e15053 100644 --- a/pkg/storage/endpoint/cluster_id_test.go +++ b/pkg/storage/endpoint/cluster_id_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/endpoint/config.go b/pkg/storage/endpoint/config.go index d297562f275..0c96de0a8cc 100644 --- a/pkg/storage/endpoint/config.go +++ b/pkg/storage/endpoint/config.go @@ -18,9 +18,10 @@ import ( "encoding/json" "strings" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) // ConfigStorage defines the storage operations on the config. diff --git a/pkg/storage/endpoint/gc_safe_point.go b/pkg/storage/endpoint/gc_safe_point.go index 8a6c5c5f789..15d3b3c47a1 100644 --- a/pkg/storage/endpoint/gc_safe_point.go +++ b/pkg/storage/endpoint/gc_safe_point.go @@ -20,11 +20,13 @@ import ( "strconv" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) // ServiceSafePoint is the safepoint for a specific service diff --git a/pkg/storage/endpoint/keyspace.go b/pkg/storage/endpoint/keyspace.go index 77240541951..e1c7ae4c846 100644 --- a/pkg/storage/endpoint/keyspace.go +++ b/pkg/storage/endpoint/keyspace.go @@ -19,11 +19,13 @@ import ( "strconv" "github.com/gogo/protobuf/proto" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/pkg/storage/endpoint/meta.go b/pkg/storage/endpoint/meta.go index 176188be3f3..73b7c397ff4 100644 --- a/pkg/storage/endpoint/meta.go +++ b/pkg/storage/endpoint/meta.go @@ -21,8 +21,10 @@ import ( "time" "github.com/gogo/protobuf/proto" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/storage/endpoint/resource_group.go b/pkg/storage/endpoint/resource_group.go index 733e5ba2c9a..431c869ac70 100644 --- a/pkg/storage/endpoint/resource_group.go +++ b/pkg/storage/endpoint/resource_group.go @@ -16,6 +16,7 @@ package endpoint import ( "github.com/gogo/protobuf/proto" + "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/endpoint/safepoint_v2.go b/pkg/storage/endpoint/safepoint_v2.go index f16855e86d6..12659847510 100644 --- a/pkg/storage/endpoint/safepoint_v2.go +++ b/pkg/storage/endpoint/safepoint_v2.go @@ -19,12 +19,14 @@ import ( "math" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // GCSafePointV2 represents the overall safe point for a specific keyspace. diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index 77841529e98..697001dfe7c 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -19,13 +19,15 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // TSOStorage is the interface for timestamp storage. diff --git a/pkg/storage/endpoint/tso_keyspace_group.go b/pkg/storage/endpoint/tso_keyspace_group.go index b45d1f1da37..ef9a23308a6 100644 --- a/pkg/storage/endpoint/tso_keyspace_group.go +++ b/pkg/storage/endpoint/tso_keyspace_group.go @@ -18,11 +18,12 @@ import ( "context" "encoding/json" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // UserKind represents the user kind. diff --git a/pkg/storage/endpoint/util.go b/pkg/storage/endpoint/util.go index 39aa6240f5a..e8f17a3922f 100644 --- a/pkg/storage/endpoint/util.go +++ b/pkg/storage/endpoint/util.go @@ -20,10 +20,11 @@ import ( "strings" "github.com/gogo/protobuf/proto" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/etcdutil" - clientv3 "go.etcd.io/etcd/client/v3" ) func (se *StorageEndpoint) loadProto(key string, msg proto.Message) (bool, error) { diff --git a/pkg/storage/etcd_backend.go b/pkg/storage/etcd_backend.go index e9af5fc67f6..b967d99b37b 100644 --- a/pkg/storage/etcd_backend.go +++ b/pkg/storage/etcd_backend.go @@ -15,9 +15,10 @@ package storage import ( + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" - clientv3 "go.etcd.io/etcd/client/v3" ) // etcdBackend is a storage backend that stores data in etcd, diff --git a/pkg/storage/hot_region_storage.go b/pkg/storage/hot_region_storage.go index d323b40d435..b5ec8dda6ba 100644 --- a/pkg/storage/hot_region_storage.go +++ b/pkg/storage/hot_region_storage.go @@ -24,12 +24,15 @@ import ( "sync" "time" - "github.com/pingcap/kvproto/pkg/encryptionpb" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/log" "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/iterator" "github.com/syndtr/goleveldb/leveldb/util" + "go.uber.org/zap" + + "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) // HotRegionStorage is used to store the hot region info. diff --git a/pkg/storage/hot_region_storage_test.go b/pkg/storage/hot_region_storage_test.go index aeda8e450e7..cd713685eb0 100644 --- a/pkg/storage/hot_region_storage_test.go +++ b/pkg/storage/hot_region_storage_test.go @@ -26,6 +26,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics/utils" ) diff --git a/pkg/storage/keyspace_test.go b/pkg/storage/keyspace_test.go index a6ce94e4711..7841f5da22d 100644 --- a/pkg/storage/keyspace_test.go +++ b/pkg/storage/keyspace_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/kv/etcd_kv.go b/pkg/storage/kv/etcd_kv.go index 5945990c51b..c25f4d66060 100644 --- a/pkg/storage/kv/etcd_kv.go +++ b/pkg/storage/kv/etcd_kv.go @@ -20,13 +20,15 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/storage/kv/kv_test.go b/pkg/storage/kv/kv_test.go index f57c8145bd0..f05561b0c0b 100644 --- a/pkg/storage/kv/kv_test.go +++ b/pkg/storage/kv/kv_test.go @@ -22,8 +22,9 @@ import ( "testing" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/etcdutil" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/pkg/utils/etcdutil" ) func TestEtcd(t *testing.T) { diff --git a/pkg/storage/kv/levedb_kv.go b/pkg/storage/kv/levedb_kv.go index 6f93cd0237f..5a74c1928e8 100644 --- a/pkg/storage/kv/levedb_kv.go +++ b/pkg/storage/kv/levedb_kv.go @@ -17,9 +17,11 @@ package kv import ( "context" - "github.com/pingcap/errors" "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/util" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/storage/kv/mem_kv.go b/pkg/storage/kv/mem_kv.go index dc24ab3e0f6..cc5dca29851 100644 --- a/pkg/storage/kv/mem_kv.go +++ b/pkg/storage/kv/mem_kv.go @@ -18,8 +18,10 @@ import ( "context" "github.com/google/btree" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/storage/leveldb_backend.go b/pkg/storage/leveldb_backend.go index 8fb1db196c1..096a762db65 100644 --- a/pkg/storage/leveldb_backend.go +++ b/pkg/storage/leveldb_backend.go @@ -18,9 +18,11 @@ import ( "context" "time" + "github.com/syndtr/goleveldb/leveldb" + "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/syndtr/goleveldb/leveldb" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/pkg/storage/leveldb_backend_test.go b/pkg/storage/leveldb_backend_test.go index c6e0fcef607..0bf626834cd 100644 --- a/pkg/storage/leveldb_backend_test.go +++ b/pkg/storage/leveldb_backend_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/pkg/storage/region_storage.go b/pkg/storage/region_storage.go index 5a581ec5155..8e6a4d87429 100644 --- a/pkg/storage/region_storage.go +++ b/pkg/storage/region_storage.go @@ -18,7 +18,9 @@ import ( "context" "github.com/gogo/protobuf/proto" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/errs" diff --git a/pkg/storage/region_storage_test.go b/pkg/storage/region_storage_test.go index f6670f8c82e..c0ae04c8ded 100644 --- a/pkg/storage/region_storage_test.go +++ b/pkg/storage/region_storage_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/pkg/storage/storage.go b/pkg/storage/storage.go index dce2f1712b8..34f2357c8f7 100644 --- a/pkg/storage/storage.go +++ b/pkg/storage/storage.go @@ -18,13 +18,15 @@ import ( "context" "sync/atomic" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - clientv3 "go.etcd.io/etcd/client/v3" ) // Storage is the interface for the backend storage of the PD. diff --git a/pkg/storage/storage_gc_test.go b/pkg/storage/storage_gc_test.go index b18fcc12afc..ffffab620b3 100644 --- a/pkg/storage/storage_gc_test.go +++ b/pkg/storage/storage_gc_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/storage/storage_test.go b/pkg/storage/storage_test.go index ca1c8e275eb..e6c12f37cbb 100644 --- a/pkg/storage/storage_test.go +++ b/pkg/storage/storage_test.go @@ -25,13 +25,15 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/keypath" - clientv3 "go.etcd.io/etcd/client/v3" ) func TestBasic(t *testing.T) { diff --git a/pkg/storage/storage_tso_test.go b/pkg/storage/storage_tso_test.go index 07ea79df47c..e8b49106a09 100644 --- a/pkg/storage/storage_tso_test.go +++ b/pkg/storage/storage_tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/syncer/client.go b/pkg/syncer/client.go index 3d7f36d9114..81652045cbf 100644 --- a/pkg/syncer/client.go +++ b/pkg/syncer/client.go @@ -19,11 +19,19 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/keepalive" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/ratelimit" @@ -31,12 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/keepalive" - "google.golang.org/grpc/status" ) const ( diff --git a/pkg/syncer/client_test.go b/pkg/syncer/client_test.go index 926be3421ab..03f18fd2b76 100644 --- a/pkg/syncer/client_test.go +++ b/pkg/syncer/client_test.go @@ -19,15 +19,17 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockserver" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/utils/grpcutil" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) // For issue https://github.com/tikv/pd/issues/3936 diff --git a/pkg/syncer/history_buffer.go b/pkg/syncer/history_buffer.go index 7ff6f202ad3..cb30eb56ee3 100644 --- a/pkg/syncer/history_buffer.go +++ b/pkg/syncer/history_buffer.go @@ -17,12 +17,14 @@ package syncer import ( "strconv" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/kv" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/syncer/history_buffer_test.go b/pkg/syncer/history_buffer_test.go index 6bcd58b1689..368cecf5a3b 100644 --- a/pkg/syncer/history_buffer_test.go +++ b/pkg/syncer/history_buffer_test.go @@ -17,8 +17,10 @@ package syncer import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/storage/kv" ) diff --git a/pkg/syncer/server.go b/pkg/syncer/server.go index 6009bba1d7d..89af3f79ccc 100644 --- a/pkg/syncer/server.go +++ b/pkg/syncer/server.go @@ -22,11 +22,16 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/ratelimit" @@ -35,9 +40,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/syncutil" - "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/pkg/systimemon/systimemon.go b/pkg/systimemon/systimemon.go index 75fc5e68d8b..e3aa67d8d7c 100644 --- a/pkg/systimemon/systimemon.go +++ b/pkg/systimemon/systimemon.go @@ -18,10 +18,12 @@ import ( "context" "time" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // StartMonitor calls systimeErrHandler if system time jump backward. diff --git a/pkg/tso/admin.go b/pkg/tso/admin.go index bc9fd1f853d..f74abb29fcb 100644 --- a/pkg/tso/admin.go +++ b/pkg/tso/admin.go @@ -19,9 +19,10 @@ import ( "net/http" "strconv" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" - "github.com/unrolled/render" ) // Handler defines the common behaviors of a basic tso handler. diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 65f61e819d1..fda5c6088ab 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -21,9 +21,13 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/member" @@ -31,8 +35,6 @@ import ( "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 2fe0df3e000..d6bf27878a7 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -24,9 +24,12 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" mcsutils "github.com/tikv/pd/pkg/mcs/utils" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/logutil" - "go.uber.org/zap" ) // Allocator is a Timestamp Oracle allocator. diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index bb5fb4587f7..d22d284e1be 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -25,11 +25,16 @@ import ( "sync" "time" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + perrors "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -47,9 +52,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index be3d53785cd..b4393a23471 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -28,9 +28,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" @@ -41,9 +46,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 0210c98626b..5293780b22f 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -21,9 +21,13 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/election" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" @@ -32,8 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const ( diff --git a/pkg/tso/util_test.go b/pkg/tso/util_test.go index f31f8781ded..c2922d49b7f 100644 --- a/pkg/tso/util_test.go +++ b/pkg/tso/util_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/keypath" ) diff --git a/pkg/unsaferecovery/unsafe_recovery_controller.go b/pkg/unsaferecovery/unsafe_recovery_controller.go index aa3cf192270..dce2069e316 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller.go @@ -23,10 +23,13 @@ import ( "strings" "time" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/btree" "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" @@ -35,7 +38,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/typeutil" - "go.uber.org/zap" ) // stage is the stage of unsafe recovery. diff --git a/pkg/unsaferecovery/unsafe_recovery_controller_test.go b/pkg/unsaferecovery/unsafe_recovery_controller_test.go index feaf5ba7430..0dab6aeca1d 100644 --- a/pkg/unsaferecovery/unsafe_recovery_controller_test.go +++ b/pkg/unsaferecovery/unsafe_recovery_controller_test.go @@ -21,11 +21,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/raft_serverpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mock/mockcluster" diff --git a/pkg/utils/apiutil/apiutil.go b/pkg/utils/apiutil/apiutil.go index 20465d8376c..a743c543468 100644 --- a/pkg/utils/apiutil/apiutil.go +++ b/pkg/utils/apiutil/apiutil.go @@ -33,13 +33,15 @@ import ( "github.com/gin-gonic/gin" "github.com/gorilla/mux" "github.com/joho/godotenv" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/slice" - "github.com/unrolled/render" - "go.uber.org/zap" ) const ( diff --git a/pkg/utils/apiutil/multiservicesapi/middleware.go b/pkg/utils/apiutil/multiservicesapi/middleware.go index 4343adcc981..5362475f9ad 100644 --- a/pkg/utils/apiutil/multiservicesapi/middleware.go +++ b/pkg/utils/apiutil/multiservicesapi/middleware.go @@ -19,11 +19,13 @@ import ( "net/url" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" - "go.uber.org/zap" ) const ( diff --git a/pkg/utils/apiutil/serverapi/middleware.go b/pkg/utils/apiutil/serverapi/middleware.go index d6fc98082d6..85b958a5554 100644 --- a/pkg/utils/apiutil/serverapi/middleware.go +++ b/pkg/utils/apiutil/serverapi/middleware.go @@ -20,15 +20,17 @@ import ( "strings" "time" + "github.com/urfave/negroni" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/urfave/negroni" - "go.uber.org/zap" ) type runtimeServiceValidator struct { diff --git a/pkg/utils/configutil/configutil.go b/pkg/utils/configutil/configutil.go index 48be7ff8c02..fe680879f73 100644 --- a/pkg/utils/configutil/configutil.go +++ b/pkg/utils/configutil/configutil.go @@ -21,8 +21,10 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/pingcap/errors" "github.com/spf13/pflag" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/encryption" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/logutil" diff --git a/pkg/utils/etcdutil/etcdutil.go b/pkg/utils/etcdutil/etcdutil.go index 70beae943f8..3e9eaa765ab 100644 --- a/pkg/utils/etcdutil/etcdutil.go +++ b/pkg/utils/etcdutil/etcdutil.go @@ -24,13 +24,6 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/utils/grpcutil" - "github.com/tikv/pd/pkg/utils/logutil" - "github.com/tikv/pd/pkg/utils/syncutil" "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/mvccpb" "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" @@ -39,6 +32,15 @@ import ( "go.etcd.io/etcd/server/v3/etcdserver" "go.uber.org/zap" "google.golang.org/grpc/codes" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/grpcutil" + "github.com/tikv/pd/pkg/utils/logutil" + "github.com/tikv/pd/pkg/utils/syncutil" ) const ( diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 92ec8967d03..43c25539207 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -28,18 +28,20 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" - "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/tikv/pd/pkg/utils/tempurl" - "github.com/tikv/pd/pkg/utils/testutil" "go.etcd.io/etcd/api/v3/etcdserverpb" "go.etcd.io/etcd/api/v3/mvccpb" etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + + "github.com/tikv/pd/pkg/utils/syncutil" + "github.com/tikv/pd/pkg/utils/tempurl" + "github.com/tikv/pd/pkg/utils/testutil" ) func TestMain(m *testing.M) { diff --git a/pkg/utils/etcdutil/health_checker.go b/pkg/utils/etcdutil/health_checker.go index d75814ec380..eafb6293238 100644 --- a/pkg/utils/etcdutil/health_checker.go +++ b/pkg/utils/etcdutil/health_checker.go @@ -21,13 +21,15 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) const pickedCountThreshold = 3 diff --git a/pkg/utils/etcdutil/testutil.go b/pkg/utils/etcdutil/testutil.go index 7cb718664d9..daca4254467 100644 --- a/pkg/utils/etcdutil/testutil.go +++ b/pkg/utils/etcdutil/testutil.go @@ -22,11 +22,12 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/utils/tempurl" - "github.com/tikv/pd/pkg/utils/testutil" "go.etcd.io/etcd/api/v3/etcdserverpb" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/server/v3/embed" + + "github.com/tikv/pd/pkg/utils/tempurl" + "github.com/tikv/pd/pkg/utils/testutil" ) // NewTestSingleConfig is used to create a etcd config for the unit test purpose. diff --git a/pkg/utils/grpcutil/grpcutil.go b/pkg/utils/grpcutil/grpcutil.go index 1d1e6478036..aea65dd32dd 100644 --- a/pkg/utils/grpcutil/grpcutil.go +++ b/pkg/utils/grpcutil/grpcutil.go @@ -23,10 +23,6 @@ import ( "strings" "time" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" - "github.com/tikv/pd/pkg/utils/logutil" "go.etcd.io/etcd/client/pkg/v3/transport" "go.uber.org/zap" "google.golang.org/grpc" @@ -35,6 +31,12 @@ import ( "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" + "github.com/tikv/pd/pkg/utils/logutil" ) const ( diff --git a/pkg/utils/grpcutil/grpcutil_test.go b/pkg/utils/grpcutil/grpcutil_test.go index fbcfe59f02c..db77a2e4002 100644 --- a/pkg/utils/grpcutil/grpcutil_test.go +++ b/pkg/utils/grpcutil/grpcutil_test.go @@ -7,10 +7,12 @@ import ( "path/filepath" "testing" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/errs" "google.golang.org/grpc/metadata" + + "github.com/pingcap/errors" + + "github.com/tikv/pd/pkg/errs" ) var ( diff --git a/pkg/utils/logutil/log.go b/pkg/utils/logutil/log.go index 4854fd7ac40..1620f9e3b70 100644 --- a/pkg/utils/logutil/log.go +++ b/pkg/utils/logutil/log.go @@ -21,10 +21,12 @@ import ( "strings" "sync/atomic" - "github.com/pingcap/log" - "github.com/tikv/pd/pkg/errs" "go.uber.org/zap" "go.uber.org/zap/zapcore" + + "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) // FileLogConfig serializes file log related config in toml/json. diff --git a/pkg/utils/metricutil/metricutil.go b/pkg/utils/metricutil/metricutil.go index 0d73c7678a8..32583248d5b 100644 --- a/pkg/utils/metricutil/metricutil.go +++ b/pkg/utils/metricutil/metricutil.go @@ -19,9 +19,11 @@ import ( "time" "unicode" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/pkg/utils/metricutil/metricutil_test.go b/pkg/utils/metricutil/metricutil_test.go index a5c183abc20..33f322d516b 100644 --- a/pkg/utils/metricutil/metricutil_test.go +++ b/pkg/utils/metricutil/metricutil_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/typeutil" ) diff --git a/pkg/utils/operatorutil/operator_check.go b/pkg/utils/operatorutil/operator_check.go index 78b9e9d9bab..4e0d4332a45 100644 --- a/pkg/utils/operatorutil/operator_check.go +++ b/pkg/utils/operatorutil/operator_check.go @@ -16,6 +16,7 @@ package operatorutil import ( "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/schedule/operator" ) diff --git a/pkg/utils/tempurl/check_env_linux.go b/pkg/utils/tempurl/check_env_linux.go index 58f902f4bb7..7e3dffc105b 100644 --- a/pkg/utils/tempurl/check_env_linux.go +++ b/pkg/utils/tempurl/check_env_linux.go @@ -18,7 +18,9 @@ package tempurl import ( "github.com/cakturk/go-netstat/netstat" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" ) diff --git a/pkg/utils/tempurl/tempurl.go b/pkg/utils/tempurl/tempurl.go index fae6f90af91..4984bb57b23 100644 --- a/pkg/utils/tempurl/tempurl.go +++ b/pkg/utils/tempurl/tempurl.go @@ -23,6 +23,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/syncutil" ) diff --git a/pkg/utils/testutil/api_check.go b/pkg/utils/testutil/api_check.go index 0b714204500..3c5ffbdbe1e 100644 --- a/pkg/utils/testutil/api_check.go +++ b/pkg/utils/testutil/api_check.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/apiutil" ) diff --git a/pkg/utils/testutil/testutil.go b/pkg/utils/testutil/testutil.go index 363b3f14aef..89e706b6b63 100644 --- a/pkg/utils/testutil/testutil.go +++ b/pkg/utils/testutil/testutil.go @@ -21,11 +21,12 @@ import ( "sync" "time" - "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/log" ) const ( diff --git a/pkg/utils/tsoutil/tso_dispatcher.go b/pkg/utils/tsoutil/tso_dispatcher.go index 86afe75e6c5..be7d4fa6d83 100644 --- a/pkg/utils/tsoutil/tso_dispatcher.go +++ b/pkg/utils/tsoutil/tso_dispatcher.go @@ -20,15 +20,17 @@ import ( "sync" "time" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/timerutil" - "go.uber.org/zap" - "google.golang.org/grpc" ) const ( diff --git a/pkg/utils/tsoutil/tso_proto_factory.go b/pkg/utils/tsoutil/tso_proto_factory.go index 9d8c0cfdb76..075a7024e98 100644 --- a/pkg/utils/tsoutil/tso_proto_factory.go +++ b/pkg/utils/tsoutil/tso_proto_factory.go @@ -17,10 +17,12 @@ package tsoutil import ( "context" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" + "github.com/tikv/pd/pkg/utils/grpcutil" - "google.golang.org/grpc" ) // ProtoFactory is the abstract factory for creating tso related data structures defined in the grpc service diff --git a/pkg/utils/tsoutil/tso_request.go b/pkg/utils/tsoutil/tso_request.go index d01247aa4b0..8a1a42a9d00 100644 --- a/pkg/utils/tsoutil/tso_request.go +++ b/pkg/utils/tsoutil/tso_request.go @@ -15,10 +15,12 @@ package tsoutil import ( + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" + "github.com/tikv/pd/pkg/mcs/utils/constant" - "google.golang.org/grpc" ) // Request is an interface wrapping tsopb.TsoRequest and pdpb.TsoRequest so diff --git a/pkg/utils/typeutil/size.go b/pkg/utils/typeutil/size.go index c088ca73e59..1356cef3588 100644 --- a/pkg/utils/typeutil/size.go +++ b/pkg/utils/typeutil/size.go @@ -18,6 +18,7 @@ import ( "strconv" "github.com/docker/go-units" + "github.com/pingcap/errors" ) diff --git a/pkg/versioninfo/feature.go b/pkg/versioninfo/feature.go index 21ad56e8585..24aed22dc9d 100644 --- a/pkg/versioninfo/feature.go +++ b/pkg/versioninfo/feature.go @@ -16,9 +16,11 @@ package versioninfo import ( "github.com/coreos/go-semver/semver" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // Feature supported features. diff --git a/pkg/versioninfo/versioninfo.go b/pkg/versioninfo/versioninfo.go index d6f4738d786..a6d7e910fb5 100644 --- a/pkg/versioninfo/versioninfo.go +++ b/pkg/versioninfo/versioninfo.go @@ -19,9 +19,11 @@ import ( "strings" "github.com/coreos/go-semver/semver" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" - "go.uber.org/zap" ) // Status is the status of PD server. diff --git a/plugin/scheduler_example/evict_leader.go b/plugin/scheduler_example/evict_leader.go index eb976edf851..4e0e700e8ce 100644 --- a/plugin/scheduler_example/evict_leader.go +++ b/plugin/scheduler_example/evict_leader.go @@ -20,8 +20,11 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/syncutil" - "github.com/unrolled/render" ) const ( diff --git a/server/api/admin.go b/server/api/admin.go index 24b9feaea75..434508c98df 100644 --- a/server/api/admin.go +++ b/server/api/admin.go @@ -22,14 +22,16 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) type adminHandler struct { diff --git a/server/api/admin_test.go b/server/api/admin_test.go index 4995c57619b..029ddce87cc 100644 --- a/server/api/admin_test.go +++ b/server/api/admin_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/replication" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/checker.go b/server/api/checker.go index 709e641c37b..adf6f7c0ccc 100644 --- a/server/api/checker.go +++ b/server/api/checker.go @@ -18,9 +18,10 @@ import ( "net/http" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type checkerHandler struct { diff --git a/server/api/cluster.go b/server/api/cluster.go index 2c29101dd2d..e6c4954ff43 100644 --- a/server/api/cluster.go +++ b/server/api/cluster.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type clusterHandler struct { diff --git a/server/api/cluster_test.go b/server/api/cluster_test.go index 0197489ecad..48854ac3fe8 100644 --- a/server/api/cluster_test.go +++ b/server/api/cluster_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/config.go b/server/api/config.go index 511f47284a9..3eda889507f 100644 --- a/server/api/config.go +++ b/server/api/config.go @@ -24,9 +24,12 @@ import ( "strings" "time" + "github.com/unrolled/render" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" sc "github.com/tikv/pd/pkg/schedule/config" @@ -36,7 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - "github.com/unrolled/render" ) // This line is to ensure the package `sc` could always be imported so that diff --git a/server/api/diagnostic.go b/server/api/diagnostic.go index 23016519dee..8df7368f074 100644 --- a/server/api/diagnostic.go +++ b/server/api/diagnostic.go @@ -18,8 +18,9 @@ import ( "net/http" "github.com/gorilla/mux" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type diagnosticHandler struct { diff --git a/server/api/diagnostic_test.go b/server/api/diagnostic_test.go index c85d9a45369..87f652d3a13 100644 --- a/server/api/diagnostic_test.go +++ b/server/api/diagnostic_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/schedule/types" diff --git a/server/api/etcd_api_test.go b/server/api/etcd_api_test.go index cec4e6369e5..5dba77de09d 100644 --- a/server/api/etcd_api_test.go +++ b/server/api/etcd_api_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + tu "github.com/tikv/pd/pkg/utils/testutil" ) diff --git a/server/api/health.go b/server/api/health.go index 93f95e8745d..dcfc21175c3 100644 --- a/server/api/health.go +++ b/server/api/health.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/server" "github.com/tikv/pd/server/cluster" - "github.com/unrolled/render" ) type healthHandler struct { diff --git a/server/api/health_test.go b/server/api/health_test.go index 89a9627bc37..40a781a6e60 100644 --- a/server/api/health_test.go +++ b/server/api/health_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" ) diff --git a/server/api/hot_status.go b/server/api/hot_status.go index e4dbf96b904..07b3c165480 100644 --- a/server/api/hot_status.go +++ b/server/api/hot_status.go @@ -21,9 +21,10 @@ import ( "net/http" "strconv" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type hotStatusHandler struct { diff --git a/server/api/hot_status_test.go b/server/api/hot_status_test.go index 1ffb9dd787f..a18b64f1e98 100644 --- a/server/api/hot_status_test.go +++ b/server/api/hot_status_test.go @@ -23,6 +23,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/syndtr/goleveldb/leveldb" + "github.com/tikv/pd/pkg/schedule/handler" "github.com/tikv/pd/pkg/storage" "github.com/tikv/pd/pkg/storage/kv" diff --git a/server/api/label.go b/server/api/label.go index ead6b30ae26..58647e2a592 100644 --- a/server/api/label.go +++ b/server/api/label.go @@ -19,12 +19,14 @@ import ( "regexp" "strings" + "github.com/unrolled/render" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type labelsHandler struct { diff --git a/server/api/label_test.go b/server/api/label_test.go index 70e4ab80066..92a59302753 100644 --- a/server/api/label_test.go +++ b/server/api/label_test.go @@ -19,9 +19,11 @@ import ( "fmt" "testing" + "github.com/stretchr/testify/suite" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/keypath" diff --git a/server/api/log.go b/server/api/log.go index 3359e70b115..6c2182694c1 100644 --- a/server/api/log.go +++ b/server/api/log.go @@ -19,10 +19,12 @@ import ( "io" "net/http" + "github.com/unrolled/render" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type logHandler struct { diff --git a/server/api/log_test.go b/server/api/log_test.go index 2478f6f4e54..641d873192b 100644 --- a/server/api/log_test.go +++ b/server/api/log_test.go @@ -19,8 +19,10 @@ import ( "fmt" "testing" - "github.com/pingcap/log" "github.com/stretchr/testify/suite" + + "github.com/pingcap/log" + tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" ) diff --git a/server/api/member.go b/server/api/member.go index 5161322463a..e9d46459a91 100644 --- a/server/api/member.go +++ b/server/api/member.go @@ -21,16 +21,18 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) type memberHandler struct { diff --git a/server/api/member_test.go b/server/api/member_test.go index b56e84b2a1a..92fb7970f0d 100644 --- a/server/api/member_test.go +++ b/server/api/member_test.go @@ -24,13 +24,15 @@ import ( "strings" "testing" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" ) type memberTestSuite struct { diff --git a/server/api/middleware.go b/server/api/middleware.go index fd0d81412ea..818d0d84e1d 100644 --- a/server/api/middleware.go +++ b/server/api/middleware.go @@ -19,14 +19,16 @@ import ( "net/http" "time" + "github.com/unrolled/render" + "github.com/urfave/negroni" + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/audit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/requestutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/cluster" - "github.com/unrolled/render" - "github.com/urfave/negroni" ) // serviceMiddlewareBuilder is used to build service middleware for HTTP api diff --git a/server/api/min_resolved_ts.go b/server/api/min_resolved_ts.go index 1edf924370f..584eb649db7 100644 --- a/server/api/min_resolved_ts.go +++ b/server/api/min_resolved_ts.go @@ -20,9 +20,10 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type minResolvedTSHandler struct { diff --git a/server/api/min_resolved_ts_test.go b/server/api/min_resolved_ts_test.go index 88c01602eab..f999af3fe56 100644 --- a/server/api/min_resolved_ts_test.go +++ b/server/api/min_resolved_ts_test.go @@ -22,9 +22,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/operator.go b/server/api/operator.go index c2529240804..6bcb0699d7b 100644 --- a/server/api/operator.go +++ b/server/api/operator.go @@ -20,10 +20,11 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type operatorHandler struct { diff --git a/server/api/plugin_disable.go b/server/api/plugin_disable.go index 596cddac5d7..ea7df0577ea 100644 --- a/server/api/plugin_disable.go +++ b/server/api/plugin_disable.go @@ -20,8 +20,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type pluginHandler struct{} diff --git a/server/api/pprof.go b/server/api/pprof.go index ee4398c60b2..071dd1b2a4b 100644 --- a/server/api/pprof.go +++ b/server/api/pprof.go @@ -26,11 +26,13 @@ import ( "strconv" "time" + "github.com/unrolled/render" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "github.com/unrolled/render" - "go.uber.org/zap" ) // pprofHandler pprof handler diff --git a/server/api/pprof_test.go b/server/api/pprof_test.go index b43feeab108..1bb1668e374 100644 --- a/server/api/pprof_test.go +++ b/server/api/pprof_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" ) diff --git a/server/api/region.go b/server/api/region.go index ae25d659544..afc32d2e762 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -24,7 +24,10 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type regionHandler struct { diff --git a/server/api/region_label.go b/server/api/region_label.go index 7958bacd371..2fa6466d480 100644 --- a/server/api/region_label.go +++ b/server/api/region_label.go @@ -20,11 +20,12 @@ import ( "strconv" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type regionLabelHandler struct { diff --git a/server/api/region_label_test.go b/server/api/region_label_test.go index 1ed0997b8e7..a35f77be1cf 100644 --- a/server/api/region_label_test.go +++ b/server/api/region_label_test.go @@ -21,8 +21,10 @@ import ( "sort" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/schedule/labeler" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/region_test.go b/server/api/region_test.go index c73dc02587d..ae91b41ef5e 100644 --- a/server/api/region_test.go +++ b/server/api/region_test.go @@ -27,10 +27,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/replication_mode.go b/server/api/replication_mode.go index 4fe2ef5da09..2e5a92c2195 100644 --- a/server/api/replication_mode.go +++ b/server/api/replication_mode.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type replicationModeHandler struct { diff --git a/server/api/router.go b/server/api/router.go index 6ea52304b63..fd999211542 100644 --- a/server/api/router.go +++ b/server/api/router.go @@ -22,13 +22,15 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/audit" "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // enableFailPointAPI enable fail point API handler. diff --git a/server/api/rule.go b/server/api/rule.go index 27a95a08269..4bd062e92c9 100644 --- a/server/api/rule.go +++ b/server/api/rule.go @@ -22,11 +22,12 @@ import ( "net/url" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type ruleHandler struct { diff --git a/server/api/scheduler.go b/server/api/scheduler.go index 8922a833a07..b2d18012c89 100644 --- a/server/api/scheduler.go +++ b/server/api/scheduler.go @@ -21,16 +21,18 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/unrolled/render" + "go.uber.org/zap" + "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" - "go.uber.org/zap" ) type schedulerHandler struct { diff --git a/server/api/server.go b/server/api/server.go index 8a58669de3b..1a744635e2d 100644 --- a/server/api/server.go +++ b/server/api/server.go @@ -20,13 +20,14 @@ import ( "strings" "github.com/gorilla/mux" + "github.com/urfave/negroni" + scheapi "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" tsoapi "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/apiutil/serverapi" "github.com/tikv/pd/server" - "github.com/urfave/negroni" ) const apiPrefix = "/pd" diff --git a/server/api/server_test.go b/server/api/server_test.go index f2ff4ffb452..9b99a6c32f8 100644 --- a/server/api/server_test.go +++ b/server/api/server_test.go @@ -23,11 +23,14 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - "go.uber.org/goleak" ) var ( diff --git a/server/api/service_gc_safepoint.go b/server/api/service_gc_safepoint.go index ca29f9c352f..ab3002d1297 100644 --- a/server/api/service_gc_safepoint.go +++ b/server/api/service_gc_safepoint.go @@ -19,9 +19,10 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type serviceGCSafepointHandler struct { diff --git a/server/api/service_gc_safepoint_test.go b/server/api/service_gc_safepoint_test.go index 6325babd438..33f78570e2d 100644 --- a/server/api/service_gc_safepoint_test.go +++ b/server/api/service_gc_safepoint_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/service_middleware.go b/server/api/service_middleware.go index da40a79cbea..6f11fb58cf4 100644 --- a/server/api/service_middleware.go +++ b/server/api/service_middleware.go @@ -22,15 +22,16 @@ import ( "reflect" "strings" + "github.com/unrolled/render" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/jsonutil" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" - - "github.com/unrolled/render" ) type serviceMiddlewareHandler struct { diff --git a/server/api/service_middleware_test.go b/server/api/service_middleware_test.go index 2599ccbfb54..b91f07560aa 100644 --- a/server/api/service_middleware_test.go +++ b/server/api/service_middleware_test.go @@ -20,8 +20,10 @@ import ( "net/http" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/ratelimit" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/stats.go b/server/api/stats.go index 5aa8fcb72a6..8672d562fd4 100644 --- a/server/api/stats.go +++ b/server/api/stats.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type statsHandler struct { diff --git a/server/api/stats_test.go b/server/api/stats_test.go index 1485f9eb5af..9c219ea58fd 100644 --- a/server/api/stats_test.go +++ b/server/api/stats_test.go @@ -19,8 +19,10 @@ import ( "net/url" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/apiutil" diff --git a/server/api/status.go b/server/api/status.go index e25f5da5287..3be84c3f598 100644 --- a/server/api/status.go +++ b/server/api/status.go @@ -17,9 +17,10 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type statusHandler struct { diff --git a/server/api/status_test.go b/server/api/status_test.go index 5444fda77b4..f75259a7728 100644 --- a/server/api/status_test.go +++ b/server/api/status_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/versioninfo" ) diff --git a/server/api/store.go b/server/api/store.go index 1342645cdb9..fb3e9cfb964 100644 --- a/server/api/store.go +++ b/server/api/store.go @@ -23,10 +23,13 @@ import ( "time" "github.com/gorilla/mux" + "github.com/unrolled/render" + "github.com/pingcap/errcode" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/response" @@ -34,7 +37,6 @@ import ( "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type storeHandler struct { diff --git a/server/api/store_test.go b/server/api/store_test.go index 47c7045ae02..dd0d1e1fe8b 100644 --- a/server/api/store_test.go +++ b/server/api/store_test.go @@ -25,10 +25,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/keypath" diff --git a/server/api/trend.go b/server/api/trend.go index 6a32b875f89..74dc43c53b5 100644 --- a/server/api/trend.go +++ b/server/api/trend.go @@ -18,13 +18,14 @@ import ( "net/http" "time" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) // Trend describes the cluster's schedule trend. diff --git a/server/api/trend_test.go b/server/api/trend_test.go index 4e93d0bea93..4e4a1be1f08 100644 --- a/server/api/trend_test.go +++ b/server/api/trend_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/server/api/tso.go b/server/api/tso.go index 95096257066..eca9dc7f8b4 100644 --- a/server/api/tso.go +++ b/server/api/tso.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/server" "github.com/unrolled/render" + + "github.com/tikv/pd/server" ) type tsoHandler struct { diff --git a/server/api/unsafe_operation.go b/server/api/unsafe_operation.go index dc41ec336e3..ec3da8a25be 100644 --- a/server/api/unsafe_operation.go +++ b/server/api/unsafe_operation.go @@ -17,10 +17,11 @@ package api import ( "net/http" + "github.com/unrolled/render" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" - "github.com/unrolled/render" ) type unsafeOperationHandler struct { diff --git a/server/api/unsafe_operation_test.go b/server/api/unsafe_operation_test.go index e708a93abfc..24cf9d7bbc3 100644 --- a/server/api/unsafe_operation_test.go +++ b/server/api/unsafe_operation_test.go @@ -19,8 +19,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/unsaferecovery" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/api/version.go b/server/api/version.go index 99a8a7376e5..fdc07a8b417 100644 --- a/server/api/version.go +++ b/server/api/version.go @@ -17,8 +17,9 @@ package api import ( "net/http" - "github.com/tikv/pd/pkg/versioninfo" "github.com/unrolled/render" + + "github.com/tikv/pd/pkg/versioninfo" ) // NOTE: This type is exported by HTTP API. Please pay more attention when modifying it. diff --git a/server/api/version_test.go b/server/api/version_test.go index 46bc80537a6..5342b1d9ebb 100644 --- a/server/api/version_test.go +++ b/server/api/version_test.go @@ -22,8 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index 89d0634ce8f..0245016734c 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -22,8 +22,10 @@ import ( "time" "github.com/gin-gonic/gin" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/server" diff --git a/server/apiv2/handlers/micro_service.go b/server/apiv2/handlers/micro_service.go index fd44665530f..c7fa0dc94f2 100644 --- a/server/apiv2/handlers/micro_service.go +++ b/server/apiv2/handlers/micro_service.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/server" "github.com/tikv/pd/server/apiv2/middlewares" diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index e99e8cf55a4..99a4e55c29f 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -20,7 +20,9 @@ import ( "strings" "github.com/gin-gonic/gin" + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" diff --git a/server/apiv2/middlewares/bootstrap_checker.go b/server/apiv2/middlewares/bootstrap_checker.go index 794316d3d0f..712cf71e491 100644 --- a/server/apiv2/middlewares/bootstrap_checker.go +++ b/server/apiv2/middlewares/bootstrap_checker.go @@ -18,6 +18,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/server" ) diff --git a/server/apiv2/middlewares/redirector.go b/server/apiv2/middlewares/redirector.go index 9c2c4081175..fb6b0496bec 100644 --- a/server/apiv2/middlewares/redirector.go +++ b/server/apiv2/middlewares/redirector.go @@ -19,11 +19,13 @@ import ( "net/url" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" - "go.uber.org/zap" ) // Redirector is a middleware to redirect the request to the right place. diff --git a/server/apiv2/router.go b/server/apiv2/router.go index 781a1cca87b..612ba8932ff 100644 --- a/server/apiv2/router.go +++ b/server/apiv2/router.go @@ -19,6 +19,7 @@ import ( "net/http" "github.com/gin-gonic/gin" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 10e9bf7ff1a..5ecd787956d 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -30,11 +30,15 @@ import ( "time" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" @@ -69,8 +73,6 @@ import ( "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var ( diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 99bb60b5558..11b5743fa13 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -27,13 +27,15 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/cluster" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/constant" diff --git a/server/cluster/cluster_worker.go b/server/cluster/cluster_worker.go index c14ceff2153..c24b509549a 100644 --- a/server/cluster/cluster_worker.go +++ b/server/cluster/cluster_worker.go @@ -17,10 +17,13 @@ package cluster import ( "bytes" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -30,7 +33,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.uber.org/zap" ) // HandleRegionHeartbeat processes RegionInfo reports from client. diff --git a/server/cluster/cluster_worker_test.go b/server/cluster/cluster_worker_test.go index afc979e2b97..150afc0c8a9 100644 --- a/server/cluster/cluster_worker_test.go +++ b/server/cluster/cluster_worker_test.go @@ -18,9 +18,11 @@ import ( "context" "testing" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/storage" ) diff --git a/server/cluster/scheduling_controller.go b/server/cluster/scheduling_controller.go index b182410c016..fd9c590ef68 100644 --- a/server/cluster/scheduling_controller.go +++ b/server/cluster/scheduling_controller.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule" "github.com/tikv/pd/pkg/schedule/checker" diff --git a/server/config/config.go b/server/config/config.go index a5183f119f2..282b5264fe9 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -28,9 +28,14 @@ import ( "github.com/BurntSushi/toml" "github.com/coreos/go-semver/semver" "github.com/docker/go-units" + "github.com/spf13/pflag" + "go.etcd.io/etcd/client/pkg/v3/transport" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/errs" rm "github.com/tikv/pd/pkg/mcs/resourcemanager/server" sc "github.com/tikv/pd/pkg/schedule/config" @@ -39,9 +44,6 @@ import ( "github.com/tikv/pd/pkg/utils/metricutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.etcd.io/etcd/client/pkg/v3/transport" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) // Config is the pd server configuration. diff --git a/server/config/config_test.go b/server/config/config_test.go index 78d6d25b73e..cd9447a5824 100644 --- a/server/config/config_test.go +++ b/server/config/config_test.go @@ -26,6 +26,7 @@ import ( "github.com/BurntSushi/toml" "github.com/spf13/pflag" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/storage" diff --git a/server/config/persist_options.go b/server/config/persist_options.go index c426e9d2420..59d42383743 100644 --- a/server/config/persist_options.go +++ b/server/config/persist_options.go @@ -25,10 +25,14 @@ import ( "unsafe" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/core/storelimit" @@ -38,8 +42,6 @@ import ( "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // PersistOptions wraps all configurations that need to persist to storage and diff --git a/server/config/service_middleware_persist_options.go b/server/config/service_middleware_persist_options.go index cd67c9dd1ac..ae4c0ef9b7a 100644 --- a/server/config/service_middleware_persist_options.go +++ b/server/config/service_middleware_persist_options.go @@ -19,6 +19,7 @@ import ( "sync/atomic" "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/server/config/util.go b/server/config/util.go index 2f25e711cc7..f2bc39459fd 100644 --- a/server/config/util.go +++ b/server/config/util.go @@ -19,6 +19,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/errs" ) diff --git a/server/forward.go b/server/forward.go index 674b3e008b6..b3d0d63c81b 100644 --- a/server/forward.go +++ b/server/forward.go @@ -20,12 +20,18 @@ import ( "strings" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/grpcutil" @@ -33,10 +39,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/server/cluster" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) func forwardTSORequest( diff --git a/server/gc_service.go b/server/gc_service.go index 114482fdd39..e9e52333905 100644 --- a/server/gc_service.go +++ b/server/gc_service.go @@ -22,16 +22,18 @@ import ( "path" "strings" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tsoutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" ) // GetGCSafePointV2 return gc safe point for the given keyspace. diff --git a/server/grpc_service.go b/server/grpc_service.go index de74916900e..d10421e87d7 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -28,6 +28,13 @@ import ( "sync/atomic" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/multierr" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" @@ -35,6 +42,7 @@ import ( "github.com/pingcap/kvproto/pkg/schedulingpb" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" @@ -47,12 +55,6 @@ import ( "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/cluster" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/multierr" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/server/handler.go b/server/handler.go index 2ecf7763ce2..9fa0ca6a5c5 100644 --- a/server/handler.go +++ b/server/handler.go @@ -22,8 +22,11 @@ import ( "path/filepath" "time" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/encryption" @@ -42,7 +45,6 @@ import ( "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - "go.uber.org/zap" ) // SchedulerConfigHandlerPath is the api router path of the schedule config handler. diff --git a/server/join/join.go b/server/join/join.go index e77675f2196..26b6f98667d 100644 --- a/server/join/join.go +++ b/server/join/join.go @@ -21,15 +21,17 @@ import ( "strings" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" ) const ( diff --git a/server/join/join_test.go b/server/join/join_test.go index edbcb7a3077..b89a2ed10ac 100644 --- a/server/join/join_test.go +++ b/server/join/join_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" diff --git a/server/keyspace_service.go b/server/keyspace_service.go index 967457198a9..e110742aa45 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -20,13 +20,15 @@ import ( "time" "github.com/gogo/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" - "go.etcd.io/etcd/api/v3/mvccpb" - clientv3 "go.etcd.io/etcd/client/v3" ) // KeyspaceServer wraps GrpcServer to provide keyspace service. diff --git a/server/server.go b/server/server.go index 7df3da153ba..a2cc32db9dd 100644 --- a/server/server.go +++ b/server/server.go @@ -32,6 +32,13 @@ import ( "github.com/coreos/go-semver/semver" "github.com/gogo/protobuf/proto" "github.com/gorilla/mux" + "go.etcd.io/etcd/api/v3/mvccpb" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -41,6 +48,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/pingcap/sysutil" + "github.com/tikv/pd/pkg/audit" bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/cgroup" @@ -53,8 +61,6 @@ import ( ms_server "github.com/tikv/pd/pkg/mcs/metastorage/server" "github.com/tikv/pd/pkg/mcs/registry" rm_server "github.com/tikv/pd/pkg/mcs/resourcemanager/server" - _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1" // init API group - _ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/ratelimit" @@ -80,12 +86,9 @@ import ( "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" - "go.etcd.io/etcd/api/v3/mvccpb" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/zap" - "google.golang.org/grpc" + + _ "github.com/tikv/pd/pkg/mcs/resourcemanager/server/apis/v1" // init API group + _ "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" // init tso API group ) const ( diff --git a/server/server_test.go b/server/server_test.go index 7dd91b9f61f..23da2078cb2 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -25,15 +25,16 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" - etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/server/testutil.go b/server/testutil.go index 2c844bf3d6b..d8bb9bb1cd9 100644 --- a/server/testutil.go +++ b/server/testutil.go @@ -23,8 +23,11 @@ import ( "sync" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/server/v3/embed" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" @@ -33,7 +36,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - "go.etcd.io/etcd/server/v3/embed" ) // NewTestServer creates a pd server for testing. diff --git a/server/util.go b/server/util.go index 0b396b0f5df..1764e4e9850 100644 --- a/server/util.go +++ b/server/util.go @@ -22,16 +22,18 @@ import ( "github.com/coreos/go-semver/semver" "github.com/gorilla/mux" + "github.com/urfave/negroni" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server/config" - "github.com/urfave/negroni" - "go.uber.org/zap" ) // CheckAndGetPDVersion checks and returns the PD version. diff --git a/tests/autoscaling/autoscaling_test.go b/tests/autoscaling/autoscaling_test.go index 10d10378323..c00bc00afe0 100644 --- a/tests/autoscaling/autoscaling_test.go +++ b/tests/autoscaling/autoscaling_test.go @@ -21,9 +21,10 @@ import ( "testing" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/cluster.go b/tests/cluster.go index 21807d2aadc..a4f445155e1 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -22,10 +22,13 @@ import ( "time" "github.com/coreos/go-semver/semver" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/autoscaling" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/dashboard" @@ -46,7 +49,6 @@ import ( "github.com/tikv/pd/server/cluster" "github.com/tikv/pd/server/config" "github.com/tikv/pd/server/join" - clientv3 "go.etcd.io/etcd/client/v3" ) // TestServer states. diff --git a/tests/compatibility/version_upgrade_test.go b/tests/compatibility/version_upgrade_test.go index 27eb07b0da2..7b5fd05071b 100644 --- a/tests/compatibility/version_upgrade_test.go +++ b/tests/compatibility/version_upgrade_test.go @@ -19,10 +19,12 @@ import ( "testing" "github.com/coreos/go-semver/semver" + "github.com/stretchr/testify/require" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/server" "github.com/tikv/pd/tests" ) diff --git a/tests/config.go b/tests/config.go index fcda28d4a3d..c1cb366dfa9 100644 --- a/tests/config.go +++ b/tests/config.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/server/config" ) diff --git a/tests/dashboard/race_test.go b/tests/dashboard/race_test.go index ad2d673f40d..6d117760f8f 100644 --- a/tests/dashboard/race_test.go +++ b/tests/dashboard/race_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/tests" ) diff --git a/tests/dashboard/service_test.go b/tests/dashboard/service_test.go index e689b1d4662..2c29230f787 100644 --- a/tests/dashboard/service_test.go +++ b/tests/dashboard/service_test.go @@ -26,11 +26,12 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/tikv/pd/pkg/dashboard" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 0462a6d9ea0..9ba63f0c83f 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -31,12 +31,18 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" @@ -56,10 +62,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/integrations/mcs" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/tests/integrations/client/client_tls_test.go b/tests/integrations/client/client_tls_test.go index fb76b4a0be1..40a36205de0 100644 --- a/tests/integrations/client/client_tls_test.go +++ b/tests/integrations/client/client_tls_test.go @@ -27,6 +27,9 @@ import ( "time" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + "google.golang.org/grpc" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -34,8 +37,6 @@ import ( "github.com/tikv/pd/pkg/utils/netutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" - "google.golang.org/grpc" ) var certScript = strings.Join([]string{".", "cert_opt.sh"}, string(filepath.Separator)) diff --git a/tests/integrations/client/gc_client_test.go b/tests/integrations/client/gc_client_test.go index 17db7345b81..0ea0c3daf19 100644 --- a/tests/integrations/client/gc_client_test.go +++ b/tests/integrations/client/gc_client_test.go @@ -20,20 +20,22 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" ) // gcClientTestReceiver is the pdpb.PD_WatchGCSafePointV2Server mock for testing. diff --git a/tests/integrations/client/global_config_test.go b/tests/integrations/client/global_config_test.go index 81aac43a293..04ef3992a2f 100644 --- a/tests/integrations/client/global_config_test.go +++ b/tests/integrations/client/global_config_test.go @@ -21,19 +21,20 @@ import ( "testing" "time" - pd "github.com/tikv/pd/client" - "github.com/tikv/pd/client/pkg/caller" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/zap" + "google.golang.org/grpc" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + + pd "github.com/tikv/pd/client" + "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server" - "go.uber.org/zap" - "google.golang.org/grpc" ) const globalConfigPath = "/global/config/" diff --git a/tests/integrations/client/http_client_test.go b/tests/integrations/client/http_client_test.go index d0ff81f9fad..0cbf12261f3 100644 --- a/tests/integrations/client/http_client_test.go +++ b/tests/integrations/client/http_client_test.go @@ -25,13 +25,15 @@ import ( "testing" "time" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/prometheus/client_golang/prometheus" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + pd "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/retry" "github.com/tikv/pd/pkg/core" diff --git a/tests/integrations/client/keyspace_test.go b/tests/integrations/client/keyspace_test.go index 841859bbb80..3c976ebbc57 100644 --- a/tests/integrations/client/keyspace_test.go +++ b/tests/integrations/client/keyspace_test.go @@ -19,8 +19,10 @@ import ( "math" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" diff --git a/tests/integrations/mcs/discovery/register_test.go b/tests/integrations/mcs/discovery/register_test.go index 147e16530b7..da6fa158307 100644 --- a/tests/integrations/mcs/discovery/register_test.go +++ b/tests/integrations/mcs/discovery/register_test.go @@ -21,13 +21,14 @@ import ( "time" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index c6b6070fda5..44347b4757d 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/tests/integrations/mcs/members/member_test.go b/tests/integrations/mcs/members/member_test.go index e1953de4e62..28275849073 100644 --- a/tests/integrations/mcs/members/member_test.go +++ b/tests/integrations/mcs/members/member_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + pdClient "github.com/tikv/pd/client/http" bs "github.com/tikv/pd/pkg/basicserver" tso "github.com/tikv/pd/pkg/mcs/tso/server" diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 22cb6b64c0a..2eb66df2e2b 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -26,11 +26,15 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "go.uber.org/zap" + "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/pkg/caller" "github.com/tikv/pd/client/resource_group/controller" @@ -39,8 +43,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" - "go.uber.org/zap" // Register Service _ "github.com/tikv/pd/pkg/mcs/registry" diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index f3e7f235018..14b867a587d 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -10,11 +10,12 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" - _ "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/handler" @@ -27,6 +28,8 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/tests" + + _ "github.com/tikv/pd/pkg/mcs/scheduling/server/apis/v1" ) type apiTestSuite struct { diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index 6a41ad0823e..d7d200814bb 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" diff --git a/tests/integrations/mcs/scheduling/meta_test.go b/tests/integrations/mcs/scheduling/meta_test.go index 0798e9f129e..4e0d5249fdb 100644 --- a/tests/integrations/mcs/scheduling/meta_test.go +++ b/tests/integrations/mcs/scheduling/meta_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/meta" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/integrations/mcs/scheduling/rule_test.go b/tests/integrations/mcs/scheduling/rule_test.go index e8b394ec075..a137619afbf 100644 --- a/tests/integrations/mcs/scheduling/rule_test.go +++ b/tests/integrations/mcs/scheduling/rule_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index 085b87afe86..d3850e4667c 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -23,11 +23,14 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/operator" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/server" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/server/api" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/mcs/testutil.go b/tests/integrations/mcs/testutil.go index d9dc7def3d1..b605462c044 100644 --- a/tests/integrations/mcs/testutil.go +++ b/tests/integrations/mcs/testutil.go @@ -19,6 +19,7 @@ import ( "sync" "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index 7abf1503bec..91614530ef1 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -24,9 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + tso "github.com/tikv/pd/pkg/mcs/tso/server" apis "github.com/tikv/pd/pkg/mcs/tso/server/apis/v1" "github.com/tikv/pd/pkg/mcs/utils/constant" diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index fc90241eb81..2c19f6588e5 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -25,10 +25,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/pdpb" + pd "github.com/tikv/pd/client" clierrs "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index a15cbb1cd03..b564076c1f0 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -24,17 +24,19 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) type tsoProxyTestSuite struct { diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index b0660ed6577..09a199c2d52 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -24,10 +24,16 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/goleak" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -45,10 +51,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/integrations/mcs" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/goleak" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) func TestMain(m *testing.M) { diff --git a/tests/integrations/realcluster/cluster.go b/tests/integrations/realcluster/cluster.go index b90ec3cb4d9..cc4f6b54713 100644 --- a/tests/integrations/realcluster/cluster.go +++ b/tests/integrations/realcluster/cluster.go @@ -23,10 +23,11 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.uber.org/zap" + + "github.com/pingcap/log" ) type clusterSuite struct { diff --git a/tests/integrations/realcluster/cluster_id_test.go b/tests/integrations/realcluster/cluster_id_test.go index b6d4bc4e74d..82233cd5b8c 100644 --- a/tests/integrations/realcluster/cluster_id_test.go +++ b/tests/integrations/realcluster/cluster_id_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" diff --git a/tests/integrations/realcluster/etcd_key_test.go b/tests/integrations/realcluster/etcd_key_test.go index ee962cb0396..10c392834af 100644 --- a/tests/integrations/realcluster/etcd_key_test.go +++ b/tests/integrations/realcluster/etcd_key_test.go @@ -21,6 +21,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/pkg/utils/testutil" ) diff --git a/tests/integrations/realcluster/mock_db.go b/tests/integrations/realcluster/mock_db.go index 2a636b9b86b..8a21bc1bb7d 100644 --- a/tests/integrations/realcluster/mock_db.go +++ b/tests/integrations/realcluster/mock_db.go @@ -20,11 +20,12 @@ import ( "github.com/DATA-DOG/go-sqlmock" mysqldriver "github.com/go-sql-driver/mysql" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "gorm.io/driver/mysql" "gorm.io/gorm" "moul.io/zapgorm2" + + "github.com/pingcap/log" ) // TestDB is a test database diff --git a/tests/integrations/realcluster/reboot_pd_test.go b/tests/integrations/realcluster/reboot_pd_test.go index d35a921cb74..1a042872b93 100644 --- a/tests/integrations/realcluster/reboot_pd_test.go +++ b/tests/integrations/realcluster/reboot_pd_test.go @@ -20,6 +20,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/client/http" ) diff --git a/tests/integrations/realcluster/scheduler_test.go b/tests/integrations/realcluster/scheduler_test.go index 8456e053bf8..0e4d10acd80 100644 --- a/tests/integrations/realcluster/scheduler_test.go +++ b/tests/integrations/realcluster/scheduler_test.go @@ -21,14 +21,16 @@ import ( "testing" "time" - "github.com/pingcap/log" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/utils/testutil" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/types" - "go.uber.org/zap" ) type schedulerSuite struct { diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index fa2cc80379d..2cda9f8734f 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -25,9 +25,11 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/opt" diff --git a/tests/integrations/tso/consistency_test.go b/tests/integrations/tso/consistency_test.go index 2ef2ebe3077..147f41a4591 100644 --- a/tests/integrations/tso/consistency_test.go +++ b/tests/integrations/tso/consistency_test.go @@ -20,10 +20,13 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" - "github.com/stretchr/testify/suite" + tso "github.com/tikv/pd/pkg/mcs/tso/server" tsopkg "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/keypath" @@ -31,7 +34,6 @@ import ( tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/tests" - "google.golang.org/grpc" ) type tsoConsistencyTestSuite struct { diff --git a/tests/integrations/tso/server_test.go b/tests/integrations/tso/server_test.go index c0d71050964..f03db197b35 100644 --- a/tests/integrations/tso/server_test.go +++ b/tests/integrations/tso/server_test.go @@ -20,16 +20,18 @@ import ( "testing" "time" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/tsopb" - "github.com/stretchr/testify/suite" + tso "github.com/tikv/pd/pkg/mcs/tso/server" tsopkg "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tempurl" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "google.golang.org/grpc" ) type tsoServerTestSuite struct { diff --git a/tests/integrations/tso/testutil.go b/tests/integrations/tso/testutil.go index 5eda641486f..25f10e24d65 100644 --- a/tests/integrations/tso/testutil.go +++ b/tests/integrations/tso/testutil.go @@ -15,8 +15,9 @@ package tso import ( - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/pdpb" ) const ( diff --git a/tests/registry/registry_test.go b/tests/registry/registry_test.go index 3551782d753..87d79c4f896 100644 --- a/tests/registry/registry_test.go +++ b/tests/registry/registry_test.go @@ -21,17 +21,19 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/interop/grpc_testing" + + "github.com/pingcap/failpoint" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/interop/grpc_testing" ) func TestMain(m *testing.M) { diff --git a/tests/scheduling_cluster.go b/tests/scheduling_cluster.go index 3f7c39eb81c..b5fc2429043 100644 --- a/tests/scheduling_cluster.go +++ b/tests/scheduling_cluster.go @@ -19,6 +19,7 @@ import ( "time" "github.com/stretchr/testify/require" + scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" sc "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/schedule/schedulers" diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index e1e4db3a26d..00c43d11309 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -27,11 +27,14 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/testutil" @@ -40,7 +43,6 @@ import ( "github.com/tikv/pd/server/api" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/api/checker_test.go b/tests/server/api/checker_test.go index 190e02ce607..14077ec30ae 100644 --- a/tests/server/api/checker_test.go +++ b/tests/server/api/checker_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" ) diff --git a/tests/server/api/operator_test.go b/tests/server/api/operator_test.go index 857ea6b3cdd..fd08a5ed556 100644 --- a/tests/server/api/operator_test.go +++ b/tests/server/api/operator_test.go @@ -24,8 +24,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/operator" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/tests/server/api/region_test.go b/tests/server/api/region_test.go index 3a187629830..b98b4419a80 100644 --- a/tests/server/api/region_test.go +++ b/tests/server/api/region_test.go @@ -22,10 +22,12 @@ import ( "strconv" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" tu "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/server/api/rule_test.go b/tests/server/api/rule_test.go index 3e8adabd4ca..3982e61b6ea 100644 --- a/tests/server/api/rule_test.go +++ b/tests/server/api/rule_test.go @@ -25,9 +25,11 @@ import ( "sync" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/tests/server/api/scheduler_test.go b/tests/server/api/scheduler_test.go index 281f7c1dc7e..1f76c469cfd 100644 --- a/tests/server/api/scheduler_test.go +++ b/tests/server/api/scheduler_test.go @@ -24,10 +24,12 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/types" "github.com/tikv/pd/pkg/slice" diff --git a/tests/server/api/testutil.go b/tests/server/api/testutil.go index 163a25c9bbb..0546bf83ee5 100644 --- a/tests/server/api/testutil.go +++ b/tests/server/api/testutil.go @@ -23,6 +23,7 @@ import ( "path" "github.com/stretchr/testify/require" + "github.com/tikv/pd/tests" ) diff --git a/tests/server/apiv2/handlers/keyspace_test.go b/tests/server/apiv2/handlers/keyspace_test.go index f3fa01851ac..ea8f14cd971 100644 --- a/tests/server/apiv2/handlers/keyspace_test.go +++ b/tests/server/apiv2/handlers/keyspace_test.go @@ -19,15 +19,17 @@ import ( "fmt" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/apiv2/handlers/testutil.go b/tests/server/apiv2/handlers/testutil.go index 1a40e8d1ac7..e5280183c52 100644 --- a/tests/server/apiv2/handlers/testutil.go +++ b/tests/server/apiv2/handlers/testutil.go @@ -21,8 +21,10 @@ import ( "io" "net/http" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/tests/server/apiv2/handlers/tso_keyspace_group_test.go b/tests/server/apiv2/handlers/tso_keyspace_group_test.go index 91a07ccd6b1..796fd514eef 100644 --- a/tests/server/apiv2/handlers/tso_keyspace_group_test.go +++ b/tests/server/apiv2/handlers/tso_keyspace_group_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server/apiv2/handlers" diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index dfdb9cb8685..b7467eb99a5 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -27,11 +27,15 @@ import ( "github.com/coreos/go-semver/semver" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/dashboard" @@ -55,8 +59,6 @@ import ( "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" "github.com/tikv/pd/tests/server/api" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) const ( diff --git a/tests/server/cluster/cluster_work_test.go b/tests/server/cluster/cluster_work_test.go index 9c3bf799116..e0ba916bf7b 100644 --- a/tests/server/cluster/cluster_work_test.go +++ b/tests/server/cluster/cluster_work_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tests/server/config/config_test.go b/tests/server/config/config_test.go index d225614fa96..bb387b68030 100644 --- a/tests/server/config/config_test.go +++ b/tests/server/config/config_test.go @@ -26,6 +26,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + cfg "github.com/tikv/pd/pkg/mcs/scheduling/server/config" "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/tests/server/id/id_test.go b/tests/server/id/id_test.go index 465259063bf..e4633c480f2 100644 --- a/tests/server/id/id_test.go +++ b/tests/server/id/id_test.go @@ -19,13 +19,15 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/join/join_fail/join_fail_test.go b/tests/server/join/join_fail/join_fail_test.go index 0e376281070..a037aa9e7b5 100644 --- a/tests/server/join/join_fail/join_fail_test.go +++ b/tests/server/join/join_fail/join_fail_test.go @@ -18,8 +18,10 @@ import ( "context" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/tests" ) diff --git a/tests/server/join/join_test.go b/tests/server/join/join_test.go index ea5eaaa35f4..8b0ce918377 100644 --- a/tests/server/join/join_test.go +++ b/tests/server/join/join_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/join" diff --git a/tests/server/keyspace/keyspace_test.go b/tests/server/keyspace/keyspace_test.go index a0175c1b727..5ce25794516 100644 --- a/tests/server/keyspace/keyspace_test.go +++ b/tests/server/keyspace/keyspace_test.go @@ -23,9 +23,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/schedule/labeler" diff --git a/tests/server/member/member_test.go b/tests/server/member/member_test.go index fd08e6557e8..cb8fd7bda00 100644 --- a/tests/server/member/member_test.go +++ b/tests/server/member/member_test.go @@ -25,17 +25,19 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/region_syncer/region_syncer_test.go b/tests/server/region_syncer/region_syncer_test.go index 80f5186f904..7edcaa5ad9b 100644 --- a/tests/server/region_syncer/region_syncer_test.go +++ b/tests/server/region_syncer/region_syncer_test.go @@ -19,14 +19,16 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/server_test.go b/tests/server/server_test.go index a5fcd33d2bc..77cd7aa5158 100644 --- a/tests/server/server_test.go +++ b/tests/server/server_test.go @@ -21,15 +21,17 @@ import ( "sync" "testing" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/storage/hot_region_storage_test.go b/tests/server/storage/hot_region_storage_test.go index 2eccb4b28e9..0643dda08d5 100644 --- a/tests/server/storage/hot_region_storage_test.go +++ b/tests/server/storage/hot_region_storage_test.go @@ -19,9 +19,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/utils" diff --git a/tests/server/tso/tso_test.go b/tests/server/tso/tso_test.go index cb6b87c83d3..51189966878 100644 --- a/tests/server/tso/tso_test.go +++ b/tests/server/tso/tso_test.go @@ -19,16 +19,18 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/tso" "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/server/watch/leader_watch_test.go b/tests/server/watch/leader_watch_test.go index 39c54b5c500..0a1a53c292d 100644 --- a/tests/server/watch/leader_watch_test.go +++ b/tests/server/watch/leader_watch_test.go @@ -19,12 +19,14 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + "go.uber.org/goleak" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" "github.com/tikv/pd/tests" - "go.uber.org/goleak" ) func TestMain(m *testing.M) { diff --git a/tests/testutil.go b/tests/testutil.go index b56fd245bd3..4f2a6beb261 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -29,11 +29,14 @@ import ( "time" "github.com/docker/go-units" + "github.com/stretchr/testify/require" + "go.uber.org/zap" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/stretchr/testify/require" + bs "github.com/tikv/pd/pkg/basicserver" "github.com/tikv/pd/pkg/core" scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" @@ -46,7 +49,6 @@ import ( "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/server" - "go.uber.org/zap" ) var ( diff --git a/tests/tso_cluster.go b/tests/tso_cluster.go index 28a52580a1f..a6e664abb5b 100644 --- a/tests/tso_cluster.go +++ b/tests/tso_cluster.go @@ -20,8 +20,10 @@ import ( "sync" "time" - "github.com/pingcap/errors" "github.com/stretchr/testify/require" + + "github.com/pingcap/errors" + tso "github.com/tikv/pd/pkg/mcs/tso/server" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" diff --git a/tools/pd-analysis/analysis/transfer_counter.go b/tools/pd-analysis/analysis/transfer_counter.go index 98d131117b4..80401c44fcc 100644 --- a/tools/pd-analysis/analysis/transfer_counter.go +++ b/tools/pd-analysis/analysis/transfer_counter.go @@ -23,8 +23,9 @@ import ( "strings" "sync" - "github.com/tikv/pd/pkg/utils/syncutil" "go.uber.org/zap" + + "github.com/tikv/pd/pkg/utils/syncutil" ) // TransferCounter is to count transfer schedule for judging whether redundant diff --git a/tools/pd-analysis/main.go b/tools/pd-analysis/main.go index 510448ddddd..3188e7707f6 100644 --- a/tools/pd-analysis/main.go +++ b/tools/pd-analysis/main.go @@ -18,9 +18,11 @@ import ( "flag" "os" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/tools/pd-analysis/analysis" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-api-bench/cases/cases.go b/tools/pd-api-bench/cases/cases.go index 118b8aaed5e..5d63b2356f6 100644 --- a/tools/pd-api-bench/cases/cases.go +++ b/tools/pd-api-bench/cases/cases.go @@ -21,12 +21,14 @@ import ( "strconv" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/opt" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-api-bench/cases/controller.go b/tools/pd-api-bench/cases/controller.go index d6bc39d6d35..c0098bced45 100644 --- a/tools/pd-api-bench/cases/controller.go +++ b/tools/pd-api-bench/cases/controller.go @@ -19,12 +19,14 @@ import ( "sync" "time" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) var base = int64(time.Second) / int64(time.Microsecond) diff --git a/tools/pd-api-bench/config/config.go b/tools/pd-api-bench/config/config.go index 47a02461cfd..c1c17624e2f 100644 --- a/tools/pd-api-bench/config/config.go +++ b/tools/pd-api-bench/config/config.go @@ -15,12 +15,14 @@ package config import ( + flag "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - flag "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/configutil" "github.com/tikv/pd/tools/pd-api-bench/cases" - "go.uber.org/zap" ) // Config is the heartbeat-bench configuration. diff --git a/tools/pd-api-bench/main.go b/tools/pd-api-bench/main.go index 2b96b0a4115..0d9235d5055 100644 --- a/tools/pd-api-bench/main.go +++ b/tools/pd-api-bench/main.go @@ -29,10 +29,16 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" - "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/prometheus/client_golang/prometheus" flag "github.com/spf13/pflag" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + pd "github.com/tikv/pd/client" pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/opt" @@ -42,10 +48,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/tools/pd-api-bench/cases" "github.com/tikv/pd/tools/pd-api-bench/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) var ( diff --git a/tools/pd-backup/main.go b/tools/pd-backup/main.go index c0ee07afd6e..e5aaad62788 100644 --- a/tools/pd-backup/main.go +++ b/tools/pd-backup/main.go @@ -21,9 +21,10 @@ import ( "strings" "time" - "github.com/tikv/pd/tools/pd-backup/pdbackup" "go.etcd.io/etcd/client/pkg/v3/transport" clientv3 "go.etcd.io/etcd/client/v3" + + "github.com/tikv/pd/tools/pd-backup/pdbackup" ) var ( diff --git a/tools/pd-backup/pdbackup/backup.go b/tools/pd-backup/pdbackup/backup.go index d02e651b879..e3fd3e682cb 100644 --- a/tools/pd-backup/pdbackup/backup.go +++ b/tools/pd-backup/pdbackup/backup.go @@ -25,11 +25,12 @@ import ( "path" "strconv" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/tools/pd-backup/pdbackup/backup_test.go b/tools/pd-backup/pdbackup/backup_test.go index d23f1878b85..3734fb6782b 100644 --- a/tools/pd-backup/pdbackup/backup_test.go +++ b/tools/pd-backup/pdbackup/backup_test.go @@ -16,15 +16,16 @@ import ( "time" "github.com/stretchr/testify/suite" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/embed" + "go.uber.org/goleak" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" - clientv3 "go.etcd.io/etcd/client/v3" - "go.etcd.io/etcd/server/v3/embed" - "go.uber.org/goleak" ) var ( diff --git a/tools/pd-backup/tests/backup_test.go b/tools/pd-backup/tests/backup_test.go index 05d2b7b92ed..7c8c03d96e0 100644 --- a/tools/pd-backup/tests/backup_test.go +++ b/tools/pd-backup/tests/backup_test.go @@ -23,9 +23,10 @@ import ( "time" "github.com/stretchr/testify/require" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/tikv/pd/tests" "github.com/tikv/pd/tools/pd-backup/pdbackup" - clientv3 "go.etcd.io/etcd/client/v3" ) func TestBackup(t *testing.T) { diff --git a/tools/pd-ctl/main.go b/tools/pd-ctl/main.go index 0d052d95680..348310f9e7a 100644 --- a/tools/pd-ctl/main.go +++ b/tools/pd-ctl/main.go @@ -20,10 +20,12 @@ import ( "os/signal" "syscall" + "go.uber.org/zap/zapcore" + "github.com/pingcap/log" + "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/pdctl/command" - "go.uber.org/zap/zapcore" ) func main() { diff --git a/tools/pd-ctl/pdctl/command/config_command.go b/tools/pd-ctl/pdctl/command/config_command.go index 126d758fe05..2e9903db550 100644 --- a/tools/pd-ctl/pdctl/command/config_command.go +++ b/tools/pd-ctl/pdctl/command/config_command.go @@ -27,6 +27,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/reflectutil" diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index c7e0cfd691d..ce79277db1b 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -24,12 +24,14 @@ import ( "os" "strings" + "github.com/spf13/cobra" + "go.etcd.io/etcd/client/pkg/v3/transport" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/spf13/cobra" + pd "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/utils/apiutil" - "go.etcd.io/etcd/client/pkg/v3/transport" ) const ( diff --git a/tools/pd-ctl/pdctl/command/hot_command.go b/tools/pd-ctl/pdctl/command/hot_command.go index 77c0ee4d7de..4a24e0d6ce4 100644 --- a/tools/pd-ctl/pdctl/command/hot_command.go +++ b/tools/pd-ctl/pdctl/command/hot_command.go @@ -22,8 +22,10 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/storage" ) diff --git a/tools/pd-ctl/pdctl/command/keyspace_command.go b/tools/pd-ctl/pdctl/command/keyspace_command.go index 93a99abc39f..2ecee481df4 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_command.go @@ -22,6 +22,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/server/apiv2/handlers" ) diff --git a/tools/pd-ctl/pdctl/command/keyspace_group_command.go b/tools/pd-ctl/pdctl/command/keyspace_group_command.go index f315417e555..ec8ec120459 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_group_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_group_command.go @@ -23,6 +23,7 @@ import ( "strings" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" ) diff --git a/tools/pd-ctl/pdctl/command/label_command.go b/tools/pd-ctl/pdctl/command/label_command.go index 6d95465392f..d6360709cea 100644 --- a/tools/pd-ctl/pdctl/command/label_command.go +++ b/tools/pd-ctl/pdctl/command/label_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" sc "github.com/tikv/pd/pkg/schedule/config" diff --git a/tools/pd-ctl/pdctl/command/log_command.go b/tools/pd-ctl/pdctl/command/log_command.go index 56c4438a6c3..086299433dd 100644 --- a/tools/pd-ctl/pdctl/command/log_command.go +++ b/tools/pd-ctl/pdctl/command/log_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/utils/apiutil" ) diff --git a/tools/pd-ctl/pdctl/command/operator.go b/tools/pd-ctl/pdctl/command/operator.go index 4e7771580de..8e3fcc3ead6 100644 --- a/tools/pd-ctl/pdctl/command/operator.go +++ b/tools/pd-ctl/pdctl/command/operator.go @@ -19,8 +19,9 @@ import ( "net/http" "strconv" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/plugin_command.go b/tools/pd-ctl/pdctl/command/plugin_command.go index 9716d1d7755..de530904477 100644 --- a/tools/pd-ctl/pdctl/command/plugin_command.go +++ b/tools/pd-ctl/pdctl/command/plugin_command.go @@ -20,6 +20,7 @@ import ( "net/http" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/schedule" ) diff --git a/tools/pd-ctl/pdctl/command/region_command.go b/tools/pd-ctl/pdctl/command/region_command.go index 3536b01a606..743d1e1f4c7 100644 --- a/tools/pd-ctl/pdctl/command/region_command.go +++ b/tools/pd-ctl/pdctl/command/region_command.go @@ -26,9 +26,10 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" "github.com/spf13/pflag" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/scheduler.go b/tools/pd-ctl/pdctl/command/scheduler.go index b8e05604b16..5dc05aff62f 100644 --- a/tools/pd-ctl/pdctl/command/scheduler.go +++ b/tools/pd-ctl/pdctl/command/scheduler.go @@ -24,8 +24,9 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/spf13/cobra" + + "github.com/pingcap/errors" ) var ( diff --git a/tools/pd-ctl/pdctl/command/store_command.go b/tools/pd-ctl/pdctl/command/store_command.go index 0ad56e0ac72..fc398902508 100644 --- a/tools/pd-ctl/pdctl/command/store_command.go +++ b/tools/pd-ctl/pdctl/command/store_command.go @@ -23,11 +23,13 @@ import ( "strconv" "strings" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/spf13/cobra" - "github.com/tikv/pd/pkg/response" "golang.org/x/text/cases" "golang.org/x/text/language" + + "github.com/pingcap/kvproto/pkg/metapb" + + "github.com/tikv/pd/pkg/response" ) var ( diff --git a/tools/pd-ctl/pdctl/command/tso_command.go b/tools/pd-ctl/pdctl/command/tso_command.go index 689420854ee..7a4e8ec6d6e 100644 --- a/tools/pd-ctl/pdctl/command/tso_command.go +++ b/tools/pd-ctl/pdctl/command/tso_command.go @@ -18,6 +18,7 @@ import ( "strconv" "github.com/spf13/cobra" + "github.com/tikv/pd/pkg/utils/tsoutil" ) diff --git a/tools/pd-ctl/pdctl/ctl.go b/tools/pd-ctl/pdctl/ctl.go index 77f1601c8f5..e975bbf7a31 100644 --- a/tools/pd-ctl/pdctl/ctl.go +++ b/tools/pd-ctl/pdctl/ctl.go @@ -24,6 +24,7 @@ import ( shellwords "github.com/mattn/go-shellwords" "github.com/spf13/cobra" "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/versioninfo" "github.com/tikv/pd/tools/pd-ctl/pdctl/command" ) diff --git a/tools/pd-ctl/tests/cluster/cluster_test.go b/tools/pd-ctl/tests/cluster/cluster_test.go index 46681da9319..8f087bdb094 100644 --- a/tools/pd-ctl/tests/cluster/cluster_test.go +++ b/tools/pd-ctl/tests/cluster/cluster_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + clusterpkg "github.com/tikv/pd/server/cluster" pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" diff --git a/tools/pd-ctl/tests/completion/completion_test.go b/tools/pd-ctl/tests/completion/completion_test.go index cf4717a26aa..da043d2727d 100644 --- a/tools/pd-ctl/tests/completion/completion_test.go +++ b/tools/pd-ctl/tests/completion/completion_test.go @@ -18,6 +18,7 @@ import ( "testing" "github.com/stretchr/testify/require" + ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" ) diff --git a/tools/pd-ctl/tests/config/config_test.go b/tools/pd-ctl/tests/config/config_test.go index 6dc7e9fb269..cf9e4163457 100644 --- a/tools/pd-ctl/tests/config/config_test.go +++ b/tools/pd-ctl/tests/config/config_test.go @@ -26,10 +26,12 @@ import ( "time" "github.com/coreos/go-semver/semver" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/ratelimit" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/placement" diff --git a/tools/pd-ctl/tests/global_test.go b/tools/pd-ctl/tests/global_test.go index 766e357088e..00987f2a8a1 100644 --- a/tools/pd-ctl/tests/global_test.go +++ b/tools/pd-ctl/tests/global_test.go @@ -21,8 +21,10 @@ import ( "net/http" "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/assertutil" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/health/health_test.go b/tools/pd-ctl/tests/health/health_test.go index be9d5027988..c3b9e9979a4 100644 --- a/tools/pd-ctl/tests/health/health_test.go +++ b/tools/pd-ctl/tests/health/health_test.go @@ -24,6 +24,8 @@ import ( "testing" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + "github.com/tikv/pd/pkg/utils/grpcutil" "github.com/tikv/pd/server/api" "github.com/tikv/pd/server/cluster" @@ -31,7 +33,6 @@ import ( pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" ) func TestHealth(t *testing.T) { diff --git a/tools/pd-ctl/tests/helper.go b/tools/pd-ctl/tests/helper.go index bdacae48c22..8e5963440ee 100644 --- a/tools/pd-ctl/tests/helper.go +++ b/tools/pd-ctl/tests/helper.go @@ -20,6 +20,7 @@ import ( "github.com/spf13/cobra" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/tools/pd-ctl/tests/hot/hot_test.go b/tools/pd-ctl/tests/hot/hot_test.go index e12b6a39a60..d01f861d861 100644 --- a/tools/pd-ctl/tests/hot/hot_test.go +++ b/tools/pd-ctl/tests/hot/hot_test.go @@ -22,10 +22,12 @@ import ( "time" "github.com/docker/go-units" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/handler" "github.com/tikv/pd/pkg/statistics" diff --git a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go index 9c16b0751f6..fca00f2fd3c 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go @@ -22,8 +22,10 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/keyspace/keyspace_test.go b/tools/pd-ctl/tests/keyspace/keyspace_test.go index 4aa3be1d21c..23a1148cd66 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_test.go @@ -22,10 +22,12 @@ import ( "strings" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/utils/testutil" diff --git a/tools/pd-ctl/tests/label/label_test.go b/tools/pd-ctl/tests/label/label_test.go index 057c9d9d9bb..b4fcbfc0e2d 100644 --- a/tools/pd-ctl/tests/label/label_test.go +++ b/tools/pd-ctl/tests/label/label_test.go @@ -21,8 +21,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server/config" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/log/log_test.go b/tools/pd-ctl/tests/log/log_test.go index 274bc4ce7df..300b016f37b 100644 --- a/tools/pd-ctl/tests/log/log_test.go +++ b/tools/pd-ctl/tests/log/log_test.go @@ -19,8 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" diff --git a/tools/pd-ctl/tests/member/member_test.go b/tools/pd-ctl/tests/member/member_test.go index dd3e465ae38..0f7d574945f 100644 --- a/tools/pd-ctl/tests/member/member_test.go +++ b/tools/pd-ctl/tests/member/member_test.go @@ -20,8 +20,10 @@ import ( "fmt" "testing" - "github.com/pingcap/kvproto/pkg/pdpb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/operator/operator_test.go b/tools/pd-ctl/tests/operator/operator_test.go index 91e97c66dbd..9e8c374ea49 100644 --- a/tools/pd-ctl/tests/operator/operator_test.go +++ b/tools/pd-ctl/tests/operator/operator_test.go @@ -22,8 +22,10 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/suite" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/server/config" diff --git a/tools/pd-ctl/tests/region/region_test.go b/tools/pd-ctl/tests/region/region_test.go index 03a1c04ef19..06077a6184a 100644 --- a/tools/pd-ctl/tests/region/region_test.go +++ b/tools/pd-ctl/tests/region/region_test.go @@ -20,9 +20,11 @@ import ( "testing" "time" + "github.com/stretchr/testify/require" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/response" "github.com/tikv/pd/server/api" diff --git a/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go b/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go index 3da72244215..ceab3b07ab7 100644 --- a/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go +++ b/tools/pd-ctl/tests/resourcemanager/resource_manager_command_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/mcs/resourcemanager/server" "github.com/tikv/pd/pkg/utils/typeutil" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/safepoint/safepoint_test.go b/tools/pd-ctl/tests/safepoint/safepoint_test.go index 9a9c54460bd..5a7ba8a2bc8 100644 --- a/tools/pd-ctl/tests/safepoint/safepoint_test.go +++ b/tools/pd-ctl/tests/safepoint/safepoint_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/server/api" pdTests "github.com/tikv/pd/tests" diff --git a/tools/pd-ctl/tests/scheduler/scheduler_test.go b/tools/pd-ctl/tests/scheduler/scheduler_test.go index 3450e59d178..787bdaa4521 100644 --- a/tools/pd-ctl/tests/scheduler/scheduler_test.go +++ b/tools/pd-ctl/tests/scheduler/scheduler_test.go @@ -22,11 +22,13 @@ import ( "testing" "time" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/spf13/cobra" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/slice" diff --git a/tools/pd-ctl/tests/store/store_test.go b/tools/pd-ctl/tests/store/store_test.go index 6f704a25e8c..79a83325f8b 100644 --- a/tools/pd-ctl/tests/store/store_test.go +++ b/tools/pd-ctl/tests/store/store_test.go @@ -24,8 +24,11 @@ import ( "testing" "time" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + "go.etcd.io/etcd/client/pkg/v3/transport" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/response" @@ -35,7 +38,6 @@ import ( pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" - "go.etcd.io/etcd/client/pkg/v3/transport" ) func TestStoreLimitV2(t *testing.T) { diff --git a/tools/pd-ctl/tests/tso/tso_test.go b/tools/pd-ctl/tests/tso/tso_test.go index 63816c40e7a..30fdda96438 100644 --- a/tools/pd-ctl/tests/tso/tso_test.go +++ b/tools/pd-ctl/tests/tso/tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" ) diff --git a/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go b/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go index da89749c2f2..652645e1570 100644 --- a/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go +++ b/tools/pd-ctl/tests/unsafe/unsafe_operation_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/stretchr/testify/require" + pdTests "github.com/tikv/pd/tests" ctl "github.com/tikv/pd/tools/pd-ctl/pdctl" "github.com/tikv/pd/tools/pd-ctl/tests" diff --git a/tools/pd-heartbeat-bench/config/config.go b/tools/pd-heartbeat-bench/config/config.go index dc5a2a6a047..41c14845074 100644 --- a/tools/pd-heartbeat-bench/config/config.go +++ b/tools/pd-heartbeat-bench/config/config.go @@ -4,11 +4,13 @@ import ( "sync/atomic" "github.com/BurntSushi/toml" + flag "github.com/spf13/pflag" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/log" - flag "github.com/spf13/pflag" + "github.com/tikv/pd/pkg/utils/configutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-heartbeat-bench/main.go b/tools/pd-heartbeat-bench/main.go index 7150c81537a..60b1db6734d 100644 --- a/tools/pd-heartbeat-bench/main.go +++ b/tools/pd-heartbeat-bench/main.go @@ -34,11 +34,15 @@ import ( "github.com/gin-contrib/gzip" "github.com/gin-contrib/pprof" "github.com/gin-gonic/gin" + "github.com/spf13/pflag" + "go.etcd.io/etcd/pkg/v3/report" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - "github.com/spf13/pflag" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" @@ -48,8 +52,6 @@ import ( "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/tools/pd-heartbeat-bench/config" "github.com/tikv/pd/tools/pd-heartbeat-bench/metrics" - "go.etcd.io/etcd/pkg/v3/report" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-heartbeat-bench/metrics/util.go b/tools/pd-heartbeat-bench/metrics/util.go index 9a61feee420..5e709630e27 100644 --- a/tools/pd-heartbeat-bench/metrics/util.go +++ b/tools/pd-heartbeat-bench/metrics/util.go @@ -22,12 +22,13 @@ import ( "strings" "time" - "github.com/pingcap/log" "github.com/prometheus/client_golang/api" v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "go.etcd.io/etcd/pkg/v3/report" "go.uber.org/zap" + + "github.com/pingcap/log" ) var ( diff --git a/tools/pd-recover/main.go b/tools/pd-recover/main.go index 3423d06bb9f..bd5ebf76d74 100644 --- a/tools/pd-recover/main.go +++ b/tools/pd-recover/main.go @@ -24,12 +24,14 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( diff --git a/tools/pd-simulator/main.go b/tools/pd-simulator/main.go index 609ed3b06e0..59a58f0a00b 100644 --- a/tools/pd-simulator/main.go +++ b/tools/pd-simulator/main.go @@ -23,8 +23,11 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/pingcap/log" flag "github.com/spf13/pflag" + "go.uber.org/zap" + + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/utils/logutil" @@ -37,7 +40,6 @@ import ( "github.com/tikv/pd/tools/pd-simulator/simulator/cases" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) var ( diff --git a/tools/pd-simulator/simulator/cases/balance_leader.go b/tools/pd-simulator/simulator/cases/balance_leader.go index a6790548dc1..f0ed73a4a9c 100644 --- a/tools/pd-simulator/simulator/cases/balance_leader.go +++ b/tools/pd-simulator/simulator/cases/balance_leader.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/balance_region.go b/tools/pd-simulator/simulator/cases/balance_region.go index d4ef7ad986f..8c7bbbe19d8 100644 --- a/tools/pd-simulator/simulator/cases/balance_region.go +++ b/tools/pd-simulator/simulator/cases/balance_region.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/cases.go b/tools/pd-simulator/simulator/cases/cases.go index 026e095342b..c1de896074f 100644 --- a/tools/pd-simulator/simulator/cases/cases.go +++ b/tools/pd-simulator/simulator/cases/cases.go @@ -16,6 +16,7 @@ package cases import ( "github.com/pingcap/kvproto/pkg/metapb" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" diff --git a/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go b/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go index 8e65feefea4..06e0582f5f8 100644 --- a/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go +++ b/tools/pd-simulator/simulator/cases/diagnose_label_isolation.go @@ -19,12 +19,14 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newLabelNotMatch1(_ *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/diagnose_rule.go b/tools/pd-simulator/simulator/cases/diagnose_rule.go index 4e7031a3a01..d1a8350dbbc 100644 --- a/tools/pd-simulator/simulator/cases/diagnose_rule.go +++ b/tools/pd-simulator/simulator/cases/diagnose_rule.go @@ -18,14 +18,16 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" + pdHttp "github.com/tikv/pd/client/http" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newRule1(_ *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/hot_read.go b/tools/pd-simulator/simulator/cases/hot_read.go index 22ff70d9312..87637063385 100644 --- a/tools/pd-simulator/simulator/cases/hot_read.go +++ b/tools/pd-simulator/simulator/cases/hot_read.go @@ -18,7 +18,9 @@ import ( "fmt" "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/hot_write.go b/tools/pd-simulator/simulator/cases/hot_write.go index adb6eb0756a..43494324936 100644 --- a/tools/pd-simulator/simulator/cases/hot_write.go +++ b/tools/pd-simulator/simulator/cases/hot_write.go @@ -18,7 +18,9 @@ import ( "fmt" "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/import_data.go b/tools/pd-simulator/simulator/cases/import_data.go index 3d329081f9e..c06e9e7dd44 100644 --- a/tools/pd-simulator/simulator/cases/import_data.go +++ b/tools/pd-simulator/simulator/cases/import_data.go @@ -21,14 +21,16 @@ import ( "github.com/docker/go-units" "github.com/go-echarts/go-echarts/charts" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) func newImportData(config *sc.SimConfig) *Case { diff --git a/tools/pd-simulator/simulator/cases/makeup_down_replica.go b/tools/pd-simulator/simulator/cases/makeup_down_replica.go index ec664e91254..d5a5021b8cf 100644 --- a/tools/pd-simulator/simulator/cases/makeup_down_replica.go +++ b/tools/pd-simulator/simulator/cases/makeup_down_replica.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/region_merge.go b/tools/pd-simulator/simulator/cases/region_merge.go index 9a278c851bd..8f19f375c8e 100644 --- a/tools/pd-simulator/simulator/cases/region_merge.go +++ b/tools/pd-simulator/simulator/cases/region_merge.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/region_split.go b/tools/pd-simulator/simulator/cases/region_split.go index 8c1f3ac7759..8299cbdf136 100644 --- a/tools/pd-simulator/simulator/cases/region_split.go +++ b/tools/pd-simulator/simulator/cases/region_split.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/scale_tikv.go b/tools/pd-simulator/simulator/cases/scale_tikv.go index 9cfe8d9dcad..49dc70e64fb 100644 --- a/tools/pd-simulator/simulator/cases/scale_tikv.go +++ b/tools/pd-simulator/simulator/cases/scale_tikv.go @@ -16,6 +16,7 @@ package cases import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/cases/stable_env.go b/tools/pd-simulator/simulator/cases/stable_env.go index 54a9f84341f..0a0c72698d8 100644 --- a/tools/pd-simulator/simulator/cases/stable_env.go +++ b/tools/pd-simulator/simulator/cases/stable_env.go @@ -16,7 +16,9 @@ package cases import ( "github.com/docker/go-units" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" diff --git a/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index 4de2ea52f88..4c84d308af7 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -23,18 +23,20 @@ import ( "sync/atomic" "time" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + pdHttp "github.com/tikv/pd/client/http" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/typeutil" sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" ) // Client is a PD (Placement Driver) client. diff --git a/tools/pd-simulator/simulator/config/config.go b/tools/pd-simulator/simulator/config/config.go index ece3b6fd91f..ead9e50e6af 100644 --- a/tools/pd-simulator/simulator/config/config.go +++ b/tools/pd-simulator/simulator/config/config.go @@ -21,6 +21,7 @@ import ( "github.com/BurntSushi/toml" "github.com/docker/go-units" + pdHttp "github.com/tikv/pd/client/http" sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/utils/configutil" diff --git a/tools/pd-simulator/simulator/conn.go b/tools/pd-simulator/simulator/conn.go index b1000c0f17b..332f46eb573 100644 --- a/tools/pd-simulator/simulator/conn.go +++ b/tools/pd-simulator/simulator/conn.go @@ -16,6 +16,7 @@ package simulator import ( "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/config" ) diff --git a/tools/pd-simulator/simulator/drive.go b/tools/pd-simulator/simulator/drive.go index c8c325cfca6..0d81a2af1ab 100644 --- a/tools/pd-simulator/simulator/drive.go +++ b/tools/pd-simulator/simulator/drive.go @@ -26,10 +26,14 @@ import ( "sync/atomic" "time" + "github.com/prometheus/client_golang/prometheus/promhttp" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/prometheus/client_golang/prometheus/promhttp" + pdHttp "github.com/tikv/pd/client/http" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" @@ -38,8 +42,6 @@ import ( "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - clientv3 "go.etcd.io/etcd/client/v3" - "go.uber.org/zap" ) // Driver promotes the cluster status change. diff --git a/tools/pd-simulator/simulator/event.go b/tools/pd-simulator/simulator/event.go index d22f35756ef..719e12f5fea 100644 --- a/tools/pd-simulator/simulator/event.go +++ b/tools/pd-simulator/simulator/event.go @@ -22,12 +22,14 @@ import ( "strconv" "sync" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) // Event affects the status of the cluster. diff --git a/tools/pd-simulator/simulator/node.go b/tools/pd-simulator/simulator/node.go index 59b0d393c47..edacf5c4129 100644 --- a/tools/pd-simulator/simulator/node.go +++ b/tools/pd-simulator/simulator/node.go @@ -22,8 +22,11 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/ratelimit" "github.com/tikv/pd/pkg/utils/syncutil" @@ -32,7 +35,6 @@ import ( sc "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/info" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-simulator/simulator/raft.go b/tools/pd-simulator/simulator/raft.go index 7f3bf78622f..13371cefdb8 100644 --- a/tools/pd-simulator/simulator/raft.go +++ b/tools/pd-simulator/simulator/raft.go @@ -17,14 +17,16 @@ package simulator import ( "context" + "go.uber.org/zap" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/syncutil" "github.com/tikv/pd/tools/pd-simulator/simulator/cases" "github.com/tikv/pd/tools/pd-simulator/simulator/config" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) // RaftEngine records all raft information. diff --git a/tools/pd-simulator/simulator/simutil/key.go b/tools/pd-simulator/simulator/simutil/key.go index a095f9d567b..ba1a19382ed 100644 --- a/tools/pd-simulator/simulator/simutil/key.go +++ b/tools/pd-simulator/simulator/simutil/key.go @@ -18,6 +18,7 @@ import ( "bytes" "github.com/pingcap/errors" + "github.com/tikv/pd/pkg/codec" ) diff --git a/tools/pd-simulator/simulator/simutil/key_test.go b/tools/pd-simulator/simulator/simutil/key_test.go index be07037501f..23cc0720b6b 100644 --- a/tools/pd-simulator/simulator/simutil/key_test.go +++ b/tools/pd-simulator/simulator/simutil/key_test.go @@ -17,8 +17,10 @@ package simutil import ( "testing" - "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/codec" "github.com/tikv/pd/pkg/core" ) diff --git a/tools/pd-simulator/simulator/simutil/logger.go b/tools/pd-simulator/simulator/simutil/logger.go index e22124f00a5..08bd0e0461f 100644 --- a/tools/pd-simulator/simulator/simutil/logger.go +++ b/tools/pd-simulator/simulator/simutil/logger.go @@ -15,8 +15,9 @@ package simutil import ( - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) // Logger is the global logger used for simulator. diff --git a/tools/pd-simulator/simulator/task.go b/tools/pd-simulator/simulator/task.go index 0921838c70b..187f08fec68 100644 --- a/tools/pd-simulator/simulator/task.go +++ b/tools/pd-simulator/simulator/task.go @@ -21,13 +21,15 @@ import ( "time" "github.com/docker/go-units" + "go.uber.org/zap" + "github.com/pingcap/kvproto/pkg/eraftpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/tools/pd-analysis/analysis" "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" - "go.uber.org/zap" ) const ( diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index ccb68a1c5fb..cd710470db5 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -29,15 +29,17 @@ import ( "time" "github.com/influxdata/tdigest" + "github.com/prometheus/client_golang/prometheus/promhttp" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" + "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/prometheus/client_golang/prometheus/promhttp" + pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/keepalive" ) const ( diff --git a/tools/pd-ut/alloc/check_env_linux.go b/tools/pd-ut/alloc/check_env_linux.go index 1a51f8075cf..c4d20bf6a65 100644 --- a/tools/pd-ut/alloc/check_env_linux.go +++ b/tools/pd-ut/alloc/check_env_linux.go @@ -18,8 +18,9 @@ package alloc import ( "github.com/cakturk/go-netstat/netstat" - "github.com/pingcap/log" "go.uber.org/zap" + + "github.com/pingcap/log" ) func environmentCheck(addr string) bool { diff --git a/tools/pd-ut/alloc/server.go b/tools/pd-ut/alloc/server.go index ffa3bce0aa5..6e7aeeb6307 100644 --- a/tools/pd-ut/alloc/server.go +++ b/tools/pd-ut/alloc/server.go @@ -23,9 +23,11 @@ import ( "time" "github.com/gin-gonic/gin" + "go.uber.org/zap" + "github.com/pingcap/log" + "github.com/tikv/pd/pkg/utils/tempurl" - "go.uber.org/zap" ) var statusAddress = flag.String("status-addr", "0.0.0.0:0", "status address") diff --git a/tools/pd-ut/alloc/tempurl.go b/tools/pd-ut/alloc/tempurl.go index 2131699133a..c7492064ac2 100644 --- a/tools/pd-ut/alloc/tempurl.go +++ b/tools/pd-ut/alloc/tempurl.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" ) diff --git a/tools/pd-ut/ut.go b/tools/pd-ut/ut.go index 7bb0cf17e9f..8efacf5933a 100644 --- a/tools/pd-ut/ut.go +++ b/tools/pd-ut/ut.go @@ -33,9 +33,10 @@ import ( "sync" "time" - "github.com/tikv/pd/tools/pd-ut/alloc" "go.uber.org/zap" + "github.com/tikv/pd/tools/pd-ut/alloc" + // Set the correct value when it runs inside docker. _ "go.uber.org/automaxprocs" ) diff --git a/tools/regions-dump/main.go b/tools/regions-dump/main.go index 5ae4241cdc0..58d19ce72f2 100644 --- a/tools/regions-dump/main.go +++ b/tools/regions-dump/main.go @@ -25,12 +25,14 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/utils/etcdutil" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( diff --git a/tools/stores-dump/main.go b/tools/stores-dump/main.go index 0409244772f..0815c6e7113 100644 --- a/tools/stores-dump/main.go +++ b/tools/stores-dump/main.go @@ -25,11 +25,13 @@ import ( "strings" "time" + "go.etcd.io/etcd/client/pkg/v3/transport" + clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/tikv/pd/pkg/utils/etcdutil" - "go.etcd.io/etcd/client/pkg/v3/transport" - clientv3 "go.etcd.io/etcd/client/v3" ) var ( From 88a761e527e54f70916b94ce2326bc7dee56a83d Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Tue, 17 Dec 2024 19:16:00 +0800 Subject: [PATCH 08/33] dep: update github.com/aws/aws-sdk-go-v2/service/kms to v1.26.3 (#8923) ref tikv/pd#4399 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 11 ++--------- tests/integrations/go.mod | 2 +- tests/integrations/go.sum | 11 ++--------- tools/go.mod | 2 +- tools/go.sum | 11 ++--------- 6 files changed, 9 insertions(+), 30 deletions(-) diff --git a/go.mod b/go.mod index 8fed2fc25fc..381d3579785 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/BurntSushi/toml v0.3.1 github.com/aws/aws-sdk-go-v2/config v1.25.12 github.com/aws/aws-sdk-go-v2/credentials v1.16.10 - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 github.com/axw/gocov v1.0.0 github.com/brianvoe/gofakeit/v6 v6.26.3 diff --git a/go.sum b/go.sum index 172a53a0909..0338557d0ab 100644 --- a/go.sum +++ b/go.sum @@ -28,7 +28,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -37,10 +36,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -49,15 +46,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/axw/gocov v1.0.0 h1:YsqYR66hUmilVr23tu8USgnJIJvnwh3n7j5zRn7x4LU= @@ -233,7 +229,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -292,8 +287,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 9d3bcea65f6..97d637697f0 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -49,7 +49,7 @@ require ( github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 // indirect diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index 2f7fd660b4f..a14e9a4ab05 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -28,7 +28,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -37,10 +36,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -49,15 +46,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -227,7 +223,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -283,8 +278,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= diff --git a/tools/go.mod b/tools/go.mod index 79bb07c91c6..7a1c7b72f36 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -60,7 +60,7 @@ require ( github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 // indirect - github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 // indirect + github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 // indirect diff --git a/tools/go.sum b/tools/go.sum index 184ab965dbb..1bfe598d503 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -26,7 +26,6 @@ github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/appleboy/gofight/v2 v2.1.2 h1:VOy3jow4vIK8BRQJoC/I9muxyYlJ2yb9ht2hZoS3rf4= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/aws/aws-sdk-go-v2 v1.17.7/go.mod h1:uzbQtefpm44goOPmdKyAlXSNcwlRgF3ePWVW6EtJvvw= github.com/aws/aws-sdk-go-v2 v1.23.5 h1:xK6C4udTyDMd82RFvNkDQxtAd00xlzFUtX4fF2nMZyg= github.com/aws/aws-sdk-go-v2 v1.23.5/go.mod h1:t3szzKfP0NeRU27uBFczDivYJjsmSnqI8kIvKyWb9ds= github.com/aws/aws-sdk-go-v2/config v1.25.12 h1:mF4cMuNh/2G+d19nWnm1vJ/ak0qK6SbqF0KtSX9pxu0= @@ -35,10 +34,8 @@ github.com/aws/aws-sdk-go-v2/credentials v1.16.10 h1:VmRkuoKaGl2ZDNGkkRQgw80Hxj1 github.com/aws/aws-sdk-go-v2/credentials v1.16.10/go.mod h1:WEn22lpd50buTs/TDqywytW5xQ2zPOMbYipIlqI6xXg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9 h1:FZVFahMyZle6WcogZCOxo6D/lkDA2lqKIn4/ueUmVXw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.9/go.mod h1:kjq7REMIkxdtcEC9/4BVXjOsNY5isz6jQbEgk6osRTU= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.31/go.mod h1:QT0BqUvX1Bh2ABdTGnjqEjvjzrCfIniM9Sc8zn9Yndo= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8 h1:8GVZIR0y6JRIUNSYI1xAMF4HDfV8H/bOsZ/8AD/uY5Q= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.8/go.mod h1:rwBfu0SoUkBUZndVgPZKAD9Y2JigaZtRP68unRiYToQ= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.25/go.mod h1:zBHOPwhBc3FlQjQJE/D3IfPWiWaQmT06Vq9aNukDo0k= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8 h1:ZE2ds/qeBkhk3yqYvS3CDCFNvd9ir5hMjlVStLZWrvM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.8/go.mod h1:/lAPPymDYL023+TS6DJmjuL42nxix2AvEvfjqOBRODk= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1 h1:uR9lXYjdPX0xY+NhvaJ4dD8rpSRz5VY81ccIIoNG+lw= @@ -47,15 +44,14 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3 h1:e3PCNeE github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.3/go.mod h1:gIeeNyaL8tIEqZrzAnTeyhHcE0yysCtcaP+N9kxLZ+E= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8 h1:EamsKe+ZjkOQjDdHd86/JCEucjFKQ9T0atWKO4s2Lgs= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.8/go.mod h1:Q0vV3/csTpbkfKLI5Sb56cJQTCTtJ0ixdb7P+Wedqiw= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8 h1:R5f4VOFi3ScTe7TtePyxLqEhNqTJIAxL57MzrXFNs6I= -github.com/aws/aws-sdk-go-v2/service/kms v1.20.8/go.mod h1:OtP3pBOgmJM+acQyQcQXtQHets3yJoVuanCx2T5M7v4= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5 h1:MRNoQVbEtjzhYFeKVMifHae4K5q4FuK9B7tTDskIF/g= +github.com/aws/aws-sdk-go-v2/service/kms v1.26.5/go.mod h1:gfe6e+rOxaiz/gr5Myk83ruBD6F9WvM7TZbLjcTNsDM= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3 h1:wKspi1zc2ZVcgZEu3k2Mt4zGKQSoZTftsoUTLsYPcVo= github.com/aws/aws-sdk-go-v2/service/sso v1.18.3/go.mod h1:zxk6y1X2KXThESWMS5CrKRvISD8mbIMab6nZrCGxDG0= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3 h1:CxAHBS0BWSUqI7qzXHc2ZpTeHaM9JNnWJ9BN6Kmo2CY= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.3/go.mod h1:7Lt5mjQ8x5rVdKqg+sKKDeuwoszDJIIPmkd8BVsEdS0= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3 h1:KfREzajmHCSYjCaMRtdLr9boUMA7KPpoPApitPlbNeo= github.com/aws/aws-sdk-go-v2/service/sts v1.26.3/go.mod h1:7Ld9eTqocTvJqqJ5K/orbSDwmGcpRdlDiLjz2DO+SL8= -github.com/aws/smithy-go v1.13.5/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.18.1 h1:pOdBTUfXNazOlxLrgeYalVnuTpKreACHtc62xLwIB3c= github.com/aws/smithy-go v1.18.1/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -226,7 +222,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -285,8 +280,6 @@ github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/ github.com/jinzhu/now v1.1.4/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/joho/godotenv v1.4.0 h1:3l4+N6zfMWnkbPEXKng2o2/MR5mSwTrBih4ZEkkz1lg= github.com/joho/godotenv v1.4.0/go.mod h1:f4LDr5Voq0i2e/R5DDNOoa2zzDfwtkZa6DnEwAbqwq4= github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= From 2d970a619a8917c35d306f401326141481c133e0 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 10:56:12 +0800 Subject: [PATCH 09/33] *: closed immediately if error rate is changed to 0 (#8887) ref tikv/pd#8678 Signed-off-by: Ryan Leung --- client/circuitbreaker/circuit_breaker.go | 27 ++- client/circuitbreaker/circuit_breaker_test.go | 18 +- client/client.go | 5 + client/metrics/metrics.go | 2 +- tests/integrations/client/client_test.go | 172 ++++++++++++++++++ 5 files changed, 205 insertions(+), 19 deletions(-) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/circuitbreaker/circuit_breaker.go index 26d1b642ea4..2c65f4f1965 100644 --- a/client/circuitbreaker/circuit_breaker.go +++ b/client/circuitbreaker/circuit_breaker.go @@ -123,6 +123,7 @@ func (cb *CircuitBreaker[T]) ChangeSettings(apply func(config *Settings)) { defer cb.mutex.Unlock() apply(cb.config) + log.Info("circuit breaker settings changed", zap.Any("config", cb.config)) } // Execute calls the given function if the CircuitBreaker is closed and returns the result of execution. @@ -238,10 +239,10 @@ func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { observedErrorRatePct := s.failureCount * 100 / total if total >= uint32(s.cb.config.ErrorRateWindow.Seconds())*s.cb.config.MinQPSForOpen && observedErrorRatePct >= s.cb.config.ErrorRateThresholdPct { // the error threshold is breached, let's move to open state and start failing all requests - log.Error("Circuit breaker tripped. Starting to fail all requests", + log.Error("circuit breaker tripped and starting to fail all requests", zap.String("name", cb.name), - zap.Uint32("observedErrorRatePct", observedErrorRatePct), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Uint32("observed-err-rate-pct", observedErrorRatePct), + zap.Any("config", cb.config)) return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen } } @@ -253,29 +254,37 @@ func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { // continue in closed state till ErrorRateWindow is over return s, nil case StateOpen: + if s.cb.config.ErrorRateThresholdPct == 0 { + return cb.newState(now, StateClosed), nil + } + if now.After(s.end) { // CoolDownInterval is over, it is time to transition to half-open state - log.Info("Circuit breaker cooldown period is over. Transitioning to half-open state to test the service", + log.Info("circuit breaker cooldown period is over. Transitioning to half-open state to test the service", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateHalfOpen), nil } else { // continue in the open state till CoolDownInterval is over return s, errs.ErrCircuitBreakerOpen } case StateHalfOpen: + if s.cb.config.ErrorRateThresholdPct == 0 { + return cb.newState(now, StateClosed), nil + } + // do we need some expire time here in case of one of pending requests is stuck forever? if s.failureCount > 0 { // there were some failures during half-open state, let's go back to open state to wait a bit longer - log.Error("Circuit breaker goes from half-open to open again as errors persist and continue to fail all requests", + log.Error("circuit breaker goes from half-open to open again as errors persist and continue to fail all requests", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateOpen), errs.ErrCircuitBreakerOpen } else if s.successCount == s.cb.config.HalfOpenSuccessCount { // all probe requests are succeeded, we can move to closed state and allow all requests - log.Info("Circuit breaker is closed. Start allowing all requests", + log.Info("circuit breaker is closed and start allowing all requests", zap.String("name", cb.name), - zap.String("config", fmt.Sprintf("%+v", cb.config))) + zap.Any("config", cb.config)) return cb.newState(now, StateClosed), nil } else if s.pendingCount < s.cb.config.HalfOpenSuccessCount { // allow more probe requests and continue in half-open state diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/circuitbreaker/circuit_breaker_test.go index 6a726028cd8..07a3c06f86e 100644 --- a/client/circuitbreaker/circuit_breaker_test.go +++ b/client/circuitbreaker/circuit_breaker_test.go @@ -38,7 +38,7 @@ var settings = Settings{ var minCountToOpen = int(settings.MinQPSForOpen * uint32(settings.ErrorRateWindow.Seconds())) -func TestCircuitBreaker_Execute_Wrapper_Return_Values(t *testing.T) { +func TestCircuitBreakerExecuteWrapperReturnValues(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) originalError := errors.New("circuit breaker is open") @@ -57,7 +57,7 @@ func TestCircuitBreaker_Execute_Wrapper_Return_Values(t *testing.T) { re.Equal(42, result) } -func TestCircuitBreaker_OpenState(t *testing.T) { +func TestCircuitBreakerOpenState(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) driveQPS(cb, minCountToOpen, Yes, re) @@ -68,7 +68,7 @@ func TestCircuitBreaker_OpenState(t *testing.T) { re.Equal(StateOpen, cb.state.stateType) } -func TestCircuitBreaker_CloseState_Not_Enough_QPS(t *testing.T) { +func TestCircuitBreakerCloseStateNotEnoughQPS(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -78,7 +78,7 @@ func TestCircuitBreaker_CloseState_Not_Enough_QPS(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_CloseState_Not_Enough_Error_Rate(t *testing.T) { +func TestCircuitBreakerCloseStateNotEnoughErrorRate(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -89,7 +89,7 @@ func TestCircuitBreaker_CloseState_Not_Enough_Error_Rate(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_Half_Open_To_Closed(t *testing.T) { +func TestCircuitBreakerHalfOpenToClosed(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -107,7 +107,7 @@ func TestCircuitBreaker_Half_Open_To_Closed(t *testing.T) { re.Equal(StateClosed, cb.state.stateType) } -func TestCircuitBreaker_Half_Open_To_Open(t *testing.T) { +func TestCircuitBreakerHalfOpenToOpen(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -130,7 +130,7 @@ func TestCircuitBreaker_Half_Open_To_Open(t *testing.T) { // in half open state, circuit breaker will allow only HalfOpenSuccessCount pending and should fast fail all other request till HalfOpenSuccessCount requests is completed // this test moves circuit breaker to the half open state and verifies that requests above HalfOpenSuccessCount are failing -func TestCircuitBreaker_Half_Open_Fail_Over_Pending_Count(t *testing.T) { +func TestCircuitBreakerHalfOpenFailOverPendingCount(t *testing.T) { re := require.New(t) cb := newCircuitBreakerMovedToHalfOpenState(re) @@ -176,7 +176,7 @@ func TestCircuitBreaker_Half_Open_Fail_Over_Pending_Count(t *testing.T) { re.Equal(uint32(1), cb.state.successCount) } -func TestCircuitBreaker_Count_Only_Requests_In_Same_Window(t *testing.T) { +func TestCircuitBreakerCountOnlyRequestsInSameWindow(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) @@ -211,7 +211,7 @@ func TestCircuitBreaker_Count_Only_Requests_In_Same_Window(t *testing.T) { re.Equal(uint32(1), cb.state.successCount) } -func TestCircuitBreaker_ChangeSettings(t *testing.T) { +func TestCircuitBreakerChangeSettings(t *testing.T) { re := require.New(t) cb := NewCircuitBreaker[int]("test_cb", AlwaysClosedSettings) diff --git a/client/client.go b/client/client.go index 31bc72c0a77..0e48707cd8d 100644 --- a/client/client.go +++ b/client/client.go @@ -25,6 +25,8 @@ import ( "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -661,6 +663,9 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio } resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { region, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) + failpoint.Inject("triggerCircuitBreaker", func() { + err = status.Error(codes.ResourceExhausted, "resource exhausted") + }) return region, isOverloaded(err), err }) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { diff --git a/client/metrics/metrics.go b/client/metrics/metrics.go index 3a3199c74a6..67268c826f5 100644 --- a/client/metrics/metrics.go +++ b/client/metrics/metrics.go @@ -152,7 +152,7 @@ func initMetrics(constLabels prometheus.Labels) { Namespace: "pd_client", Subsystem: "request", Name: "circuit_breaker_count", - Help: "Circuit Breaker counters", + Help: "Circuit breaker counters", ConstLabels: constLabels, }, []string{"name", "success"}) } diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 9ba63f0c83f..c0b762d0983 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -20,6 +20,7 @@ import ( "encoding/json" "fmt" "math" + "os" "path" "reflect" "sort" @@ -44,6 +45,7 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" pd "github.com/tikv/pd/client" + cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" @@ -2049,3 +2051,173 @@ func needRetry(err error) bool { } return st.Code() == codes.ResourceExhausted } + +func TestCircuitBreaker(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 1, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + // wait cooldown + time.Sleep(time.Second) + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } +} + +func TestCircuitBreakerOpenAndChangeSettings(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 1, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + *config = cb.AlwaysClosedSettings + }) + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "ResourceExhausted") + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) +} + +func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + + circuitBreakerSettings := cb.Settings{ + ErrorRateThresholdPct: 60, + MinQPSForOpen: 10, + ErrorRateWindow: time.Millisecond, + CoolDownInterval: time.Second, + HalfOpenSuccessCount: 20, + } + + endpoints := runServer(re, cluster) + cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + defer cli.Close() + + for range 10 { + region, err := cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + re.NotNil(region) + } + + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + } + + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.Contains(err.Error(), "circuit breaker is open") + + fname := testutil.InitTempFileLogger("info") + defer os.RemoveAll(fname) + // wait for cooldown + time.Sleep(time.Second) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + // trigger circuit breaker state to be half open + _, err = cli.GetRegion(context.TODO(), []byte("a")) + re.NoError(err) + testutil.Eventually(re, func() bool { + b, _ := os.ReadFile(fname) + l := string(b) + // We need to check the log to see if the circuit breaker is half open + return strings.Contains(l, "Transitioning to half-open state to test the service") + }) + + // The state is half open + re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + // change settings to always closed + cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + *config = cb.AlwaysClosedSettings + }) + + // It won't be changed to open state. + for range 100 { + _, err := cli.GetRegion(context.TODO(), []byte("a")) + re.Error(err) + re.NotContains(err.Error(), "circuit breaker is open") + } + re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) +} From 221877d424db768cca77bee40351cf219f75eb01 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 16:50:51 +0800 Subject: [PATCH 10/33] *: unify the error definition (#8902) ref tikv/pd#8922 Signed-off-by: Ryan Leung --- errors.toml | 155 +++++++++++++++++++++++ pkg/cgroup/cgroup_cpu.go | 6 +- pkg/cgroup/cgroup_cpu_test.go | 4 +- pkg/errs/errno.go | 75 ++++++++++- pkg/gc/safepoint_v2.go | 3 +- pkg/keyspace/keyspace.go | 43 ++++--- pkg/keyspace/tso_keyspace_group.go | 99 ++++++++------- pkg/keyspace/tso_keyspace_group_test.go | 39 +++--- pkg/keyspace/util.go | 57 --------- pkg/ratelimit/runner.go | 10 +- pkg/schedule/checker/rule_checker.go | 22 +--- pkg/schedule/scatter/region_scatterer.go | 10 +- server/apiv2/handlers/keyspace.go | 2 +- server/keyspace_service.go | 6 +- 14 files changed, 346 insertions(+), 185 deletions(-) diff --git a/errors.toml b/errors.toml index 9bfd4a79190..26b15c38077 100644 --- a/errors.toml +++ b/errors.toml @@ -61,6 +61,11 @@ error = ''' unsupported metrics type %v ''' +["PD:cgroup:ErrNoCPUControllerDetected"] +error = ''' +no cpu controller detected +''' + ["PD:checker:ErrCheckerMergeAgain"] error = ''' region will be merged again, %s @@ -71,6 +76,36 @@ error = ''' checker not found ''' +["PD:checker:ErrNoNewLeader"] +error = ''' +no new leader +''' + +["PD:checker:ErrNoStoreToAdd"] +error = ''' +no store to add peer +''' + +["PD:checker:ErrNoStoreToReplace"] +error = ''' +no store to replace peer +''' + +["PD:checker:ErrPeerCannotBeLeader"] +error = ''' +peer cannot be leader +''' + +["PD:checker:ErrPeerCannotBeWitness"] +error = ''' +peer cannot be witness +''' + +["PD:checker:ErrRegionNoLeader"] +error = ''' +region no leader +''' + ["PD:client:ErrClientCreateTSOStream"] error = ''' create TSO stream failed, %s @@ -491,6 +526,116 @@ error = ''' failed to unmarshal json ''' +["PD:keyspace:ErrExceedMaxEtcdTxnOps"] +error = ''' +exceed max etcd txn operations +''' + +["PD:keyspace:ErrIllegalOperation"] +error = ''' +unknown operation +''' + +["PD:keyspace:ErrKeyspaceExists"] +error = ''' +keyspace already exists +''' + +["PD:keyspace:ErrKeyspaceGroupExists"] +error = ''' +keyspace group already exists +''' + +["PD:keyspace:ErrKeyspaceGroupInMerging"] +error = ''' +keyspace group %v is in merging state +''' + +["PD:keyspace:ErrKeyspaceGroupInSplit"] +error = ''' +keyspace group %v is in split state +''' + +["PD:keyspace:ErrKeyspaceGroupNotEnoughReplicas"] +error = ''' +not enough replicas in the keyspace group +''' + +["PD:keyspace:ErrKeyspaceGroupNotExists"] +error = ''' +keyspace group %v does not exist +''' + +["PD:keyspace:ErrKeyspaceGroupNotInMerging"] +error = ''' +keyspace group %v is not in merging state +''' + +["PD:keyspace:ErrKeyspaceGroupNotInSplit"] +error = ''' +keyspace group %v is not in split state +''' + +["PD:keyspace:ErrKeyspaceGroupPrimaryNotFound"] +error = ''' +primary of keyspace group does not exist +''' + +["PD:keyspace:ErrKeyspaceGroupWithEmptyKeyspace"] +error = ''' +keyspace group with empty keyspace +''' + +["PD:keyspace:ErrKeyspaceNotFound"] +error = ''' +keyspace does not exist +''' + +["PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup"] +error = ''' +keyspace is not in any keyspace group +''' + +["PD:keyspace:ErrKeyspaceNotInKeyspaceGroup"] +error = ''' +keyspace is not in this keyspace group +''' + +["PD:keyspace:ErrModifyDefaultKeyspace"] +error = ''' +cannot modify default keyspace's state +''' + +["PD:keyspace:ErrModifyDefaultKeyspaceGroup"] +error = ''' +default keyspace group cannot be modified +''' + +["PD:keyspace:ErrNoAvailableNode"] +error = ''' +no available node +''' + +["PD:keyspace:ErrNodeNotInKeyspaceGroup"] +error = ''' +the tso node is not in this keyspace group +''' + +["PD:keyspace:ErrRegionSplitFailed"] +error = ''' +region split failed +''' + +["PD:keyspace:ErrRegionSplitTimeout"] +error = ''' +region split timeout +''' + +["PD:keyspace:ErrUnsupportedOperationInKeyspace"] +error = ''' +it's a unsupported operation +''' + ["PD:leveldb:ErrLevelDBClose"] error = ''' close leveldb error @@ -646,6 +791,11 @@ error = ''' failed to unmarshal proto ''' +["PD:ratelimit:ErrMaxWaitingTasksExceeded"] +error = ''' +max waiting tasks exceeded +''' + ["PD:region:ErrRegionAbnormalPeer"] error = ''' region %v has abnormal peer @@ -691,6 +841,11 @@ error = ''' invalid group settings, please check the group name, priority and the number of resources ''' +["PD:scatter:ErrEmptyRegion"] +error = ''' +empty region +''' + ["PD:schedule:ErrCreateOperator"] error = ''' unable to create operator, %s diff --git a/pkg/cgroup/cgroup_cpu.go b/pkg/cgroup/cgroup_cpu.go index 67eace5363c..e696e36fad5 100644 --- a/pkg/cgroup/cgroup_cpu.go +++ b/pkg/cgroup/cgroup_cpu.go @@ -19,9 +19,9 @@ import ( "path/filepath" "github.com/pingcap/errors" -) -var errNoCPUControllerDetected = errors.New("no cpu controller detected") + "github.com/tikv/pd/pkg/errs" +) // Helper function for getCgroupCPU. Root is always "/", except in tests. func getCgroupCPUHelper(root string) (CPUUsage, error) { @@ -32,7 +32,7 @@ func getCgroupCPUHelper(root string) (CPUUsage, error) { // No CPU controller detected if path == "" { - return CPUUsage{}, errNoCPUControllerDetected + return CPUUsage{}, errs.ErrNoCPUControllerDetected } mount, ver, err := getCgroupDetails(filepath.Join(root, procPathMountInfo), path, "cpu,cpuacct") diff --git a/pkg/cgroup/cgroup_cpu_test.go b/pkg/cgroup/cgroup_cpu_test.go index 265291163c3..441c2192e79 100644 --- a/pkg/cgroup/cgroup_cpu_test.go +++ b/pkg/cgroup/cgroup_cpu_test.go @@ -26,6 +26,8 @@ import ( "testing" "github.com/stretchr/testify/require" + + "github.com/tikv/pd/pkg/errs" ) func checkKernelVersionNewerThan(re *require.Assertions, t *testing.T, major, minor int) bool { @@ -82,7 +84,7 @@ func TestGetCgroupCPU(t *testing.T) { }() } cpu, err := GetCgroupCPU() - if err == errNoCPUControllerDetected { + if err == errs.ErrNoCPUControllerDetected { // for more information, please refer https://github.com/pingcap/tidb/pull/41347 if checkKernelVersionNewerThan(re, t, 4, 7) { re.NoError(err, "linux version > v4.7 and err still happens") diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index e5c23cffde2..fd9ff3c95ff 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -140,8 +140,69 @@ var ( // checker errors var ( - ErrCheckerNotFound = errors.Normalize("checker not found", errors.RFCCodeText("PD:checker:ErrCheckerNotFound")) - ErrCheckerMergeAgain = errors.Normalize("region will be merged again, %s", errors.RFCCodeText("PD:checker:ErrCheckerMergeAgain")) + ErrCheckerNotFound = errors.Normalize("checker not found", errors.RFCCodeText("PD:checker:ErrCheckerNotFound")) + ErrCheckerMergeAgain = errors.Normalize("region will be merged again, %s", errors.RFCCodeText("PD:checker:ErrCheckerMergeAgain")) + ErrNoStoreToAdd = errors.Normalize("no store to add peer", errors.RFCCodeText("PD:checker:ErrNoStoreToAdd")) + ErrNoStoreToReplace = errors.Normalize("no store to replace peer", errors.RFCCodeText("PD:checker:ErrNoStoreToReplace")) + ErrPeerCannotBeLeader = errors.Normalize("peer cannot be leader", errors.RFCCodeText("PD:checker:ErrPeerCannotBeLeader")) + ErrPeerCannotBeWitness = errors.Normalize("peer cannot be witness", errors.RFCCodeText("PD:checker:ErrPeerCannotBeWitness")) + ErrNoNewLeader = errors.Normalize("no new leader", errors.RFCCodeText("PD:checker:ErrNoNewLeader")) + ErrRegionNoLeader = errors.Normalize("region no leader", errors.RFCCodeText("PD:checker:ErrRegionNoLeader")) +) + +// scatter errors +var ( + ErrEmptyRegion = errors.Normalize("empty region", errors.RFCCodeText("PD:scatter:ErrEmptyRegion")) +) + +// keyspace errors +var ( + // ErrKeyspaceNotFound is used to indicate target keyspace does not exist. + ErrKeyspaceNotFound = errors.Normalize("keyspace does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotFound")) + // ErrRegionSplitTimeout indices to split region timeout + ErrRegionSplitTimeout = errors.Normalize("region split timeout", errors.RFCCodeText("PD:keyspace:ErrRegionSplitTimeout")) + // ErrRegionSplitFailed indices to split region failed + ErrRegionSplitFailed = errors.Normalize("region split failed", errors.RFCCodeText("PD:keyspace:ErrRegionSplitFailed")) + // ErrKeyspaceExists indicates target keyspace already exists. + // It's used when creating a new keyspace. + ErrKeyspaceExists = errors.Normalize("keyspace already exists", errors.RFCCodeText("PD:keyspace:ErrKeyspaceExists")) + // ErrKeyspaceGroupExists indicates target keyspace group already exists. + ErrKeyspaceGroupExists = errors.Normalize("keyspace group already exists", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupExists")) + // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. + ErrKeyspaceNotInKeyspaceGroup = errors.Normalize("keyspace is not in this keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInKeyspaceGroup")) + // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. + ErrKeyspaceNotInAnyKeyspaceGroup = errors.Normalize("keyspace is not in any keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup")) + // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. + ErrNodeNotInKeyspaceGroup = errors.Normalize("the tso node is not in this keyspace group", errors.RFCCodeText("PD:keyspace:ErrNodeNotInKeyspaceGroup")) + // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. + ErrKeyspaceGroupNotEnoughReplicas = errors.Normalize("not enough replicas in the keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotEnoughReplicas")) + // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. + ErrKeyspaceGroupWithEmptyKeyspace = errors.Normalize("keyspace group with empty keyspace", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupWithEmptyKeyspace")) + // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. + ErrModifyDefaultKeyspaceGroup = errors.Normalize("default keyspace group cannot be modified", errors.RFCCodeText("PD:keyspace:ErrModifyDefaultKeyspaceGroup")) + // ErrNoAvailableNode is used to indicate no available node in the keyspace group. + ErrNoAvailableNode = errors.Normalize("no available node", errors.RFCCodeText("PD:keyspace:ErrNoAvailableNode")) + // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. + ErrExceedMaxEtcdTxnOps = errors.Normalize("exceed max etcd txn operations", errors.RFCCodeText("PD:keyspace:ErrExceedMaxEtcdTxnOps")) + // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. + ErrModifyDefaultKeyspace = errors.Normalize("cannot modify default keyspace's state", errors.RFCCodeText("PD:keyspace:ErrModifyDefaultKeyspace")) + // ErrIllegalOperation is used to indicate this is an illegal operation. + ErrIllegalOperation = errors.Normalize("unknown operation", errors.RFCCodeText("PD:keyspace:ErrIllegalOperation")) + // ErrUnsupportedOperationInKeyspace is used to indicate this is an unsupported operation. + ErrUnsupportedOperationInKeyspace = errors.Normalize("it's a unsupported operation", errors.RFCCodeText("PD:keyspace:ErrUnsupportedOperationInKeyspace")) + // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. + ErrKeyspaceGroupPrimaryNotFound = errors.Normalize("primary of keyspace group does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupPrimaryNotFound")) + // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. + ErrKeyspaceGroupNotExists = errors.Normalize("keyspace group %v does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotExists")) + // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. + ErrKeyspaceGroupInSplit = errors.Normalize("keyspace group %v is in split state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupInSplit")) + // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. + ErrKeyspaceGroupNotInSplit = errors.Normalize("keyspace group %v is not in split state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotInSplit")) + // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. + ErrKeyspaceGroupInMerging = errors.Normalize("keyspace group %v is in merging state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupInMerging")) + // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. + ErrKeyspaceGroupNotInMerging = errors.Normalize("keyspace group %v is not in merging state", errors.RFCCodeText("PD:keyspace:ErrKeyspaceGroupNotInMerging")) + // errKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. ) // diagnostic errors @@ -229,6 +290,16 @@ var ( ErrBytesToUint64 = errors.Normalize("invalid data, must 8 bytes, but %d", errors.RFCCodeText("PD:typeutil:ErrBytesToUint64")) ) +// cgroup errors +var ( + ErrNoCPUControllerDetected = errors.Normalize("no cpu controller detected", errors.RFCCodeText("PD:cgroup:ErrNoCPUControllerDetected")) +) + +// ratelimit errors +var ( + ErrMaxWaitingTasksExceeded = errors.Normalize("max waiting tasks exceeded", errors.RFCCodeText("PD:ratelimit:ErrMaxWaitingTasksExceeded")) +) + // The third-party project error. // url errors var ( diff --git a/pkg/gc/safepoint_v2.go b/pkg/gc/safepoint_v2.go index 449be8f3d59..acdd4e6eef8 100644 --- a/pkg/gc/safepoint_v2.go +++ b/pkg/gc/safepoint_v2.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" @@ -101,7 +102,7 @@ func (manager *SafePointV2Manager) checkKeyspace(keyspaceID uint32, updateReques } // If a keyspace does not exist, then loading its gc safe point is prohibited. if meta == nil { - return keyspace.ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // If keyspace's state does not permit updating safe point, we return error. if updateRequest && !slice.Contains(allowUpdateSafePoint, meta.GetState()) { diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 4a50f36169f..c9e390df47a 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/core" @@ -153,7 +154,7 @@ func (manager *Manager) Bootstrap() error { err = manager.saveNewKeyspace(defaultKeyspaceMeta) // It's possible that default keyspace already exists in the storage (e.g. PD restart/recover), // so we ignore the keyspaceExists error. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], defaultKeyspaceMeta.GetId(), opAdd); err != nil { @@ -174,7 +175,7 @@ func (manager *Manager) Bootstrap() error { } keyspace, err := manager.CreateKeyspace(req) // Ignore the keyspaceExists error for the same reason as saving default keyspace. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], keyspace.GetId(), opAdd); err != nil { @@ -288,7 +289,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if nameExists { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } err = manager.store.SaveKeyspaceID(txn, keyspace.Id, keyspace.Name) if err != nil { @@ -301,7 +302,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if loadedMeta != nil { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } return manager.store.SaveKeyspaceMeta(txn, keyspace) }) @@ -343,7 +344,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er ranges := keyspaceRule.Data.([]*labeler.KeyRangeRule) if len(ranges) < 2 { log.Warn("[keyspace] failed to split keyspace region with insufficient range", logutil.ZapRedactString("label-rule", keyspaceRule.String())) - return ErrRegionSplitFailed + return errs.ErrRegionSplitFailed } rawLeftBound, rawRightBound := ranges[0].StartKey, ranges[0].EndKey txnLeftBound, txnRightBound := ranges[1].StartKey, ranges[1].EndKey @@ -381,7 +382,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er zap.Uint32("keyspace-id", id), zap.Error(err), ) - err = ErrRegionSplitTimeout + err = errs.ErrRegionSplitTimeout return } log.Info("[keyspace] wait region split successfully", zap.Uint32("keyspace-id", id)) @@ -407,14 +408,14 @@ func (manager *Manager) LoadKeyspace(name string) (*keyspacepb.KeyspaceMeta, err return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } meta, err = manager.store.LoadKeyspaceMeta(txn, id) if err != nil { return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -434,7 +435,7 @@ func (manager *Manager) LoadKeyspaceByID(spaceID uint32) (*keyspacepb.KeyspaceMe return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -474,7 +475,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -484,7 +485,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Only keyspace with state listed in allowChangeConfig are allowed to change their config. if !slice.Contains(allowChangeConfig, meta.GetState()) { @@ -505,7 +506,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) case OpDel: delete(meta.Config, mutation.Key) default: - return errIllegalOperation + return errs.ErrIllegalOperation } } newConfig := meta.GetConfig() @@ -553,9 +554,9 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key // Changing the state of default keyspace is not allowed. if name == constant.DefaultKeyspaceName { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), + errs.ZapError(errs.ErrModifyDefaultKeyspace), ) - return nil, ErrModifyDefaultKeyspace + return nil, errs.ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta err := manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { @@ -565,7 +566,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -575,7 +576,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -605,9 +606,9 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K // Changing the state of default keyspace is not allowed. if id == constant.DefaultKeyspaceID { log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), + errs.ZapError(errs.ErrModifyDefaultKeyspace), ) - return nil, ErrModifyDefaultKeyspace + return nil, errs.ErrModifyDefaultKeyspace } var meta *keyspacepb.KeyspaceMeta var err error @@ -620,7 +621,7 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -740,10 +741,10 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID return errors.Errorf("default keyspace group %d not found", constant.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsSplitting() { - return ErrKeyspaceGroupInSplit(constant.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(constant.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsMerging() { - return ErrKeyspaceGroupInMerging(constant.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(constant.DefaultKeyspaceGroupID) } keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, etcdutil.MaxEtcdTxnOps) if err != nil { diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index dc414cfad3a..a6068ffc2bb 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/pkg/balancer" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/slice" @@ -129,7 +130,7 @@ func (m *GroupManager) Bootstrap(ctx context.Context) error { // Ignore the error if default keyspace group already exists in the storage (e.g. PD restart/recover). err := m.saveKeyspaceGroups([]*endpoint.KeyspaceGroup{defaultKeyspaceGroup}, false) - if err != nil && err != ErrKeyspaceGroupExists { + if err != nil && err != errs.ErrKeyspaceGroupExists { return err } @@ -322,7 +323,7 @@ func (m *GroupManager) DeleteKeyspaceGroupByID(id uint32) (*endpoint.KeyspaceGro return nil } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } return m.store.DeleteKeyspaceGroup(txn, id) }); err != nil { @@ -348,13 +349,13 @@ func (m *GroupManager) saveKeyspaceGroups(keyspaceGroups []*endpoint.KeyspaceGro return err } if oldKG != nil && !overwrite { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } if oldKG.IsSplitting() && overwrite { - return ErrKeyspaceGroupInSplit(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(keyspaceGroup.ID) } if oldKG.IsMerging() && overwrite { - return ErrKeyspaceGroupInMerging(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(keyspaceGroup.ID) } newKG := &endpoint.KeyspaceGroup{ ID: keyspaceGroup.ID, @@ -410,7 +411,7 @@ func (m *GroupManager) GetGroupByKeyspaceID(id uint32) (uint32, error) { } } } - return 0, ErrKeyspaceNotInAnyKeyspaceGroup + return 0, errs.ErrKeyspaceNotInAnyKeyspaceGroup } var failpointOnce sync.Once @@ -440,13 +441,13 @@ func (m *GroupManager) UpdateKeyspaceForGroup(userKind endpoint.UserKind, groupI func (m *GroupManager) updateKeyspaceForGroupLocked(userKind endpoint.UserKind, groupID uint64, keyspaceID uint32, mutation int) error { kg := m.groups[userKind].Get(uint32(groupID)) if kg == nil { - return ErrKeyspaceGroupNotExists(uint32(groupID)) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(uint32(groupID)) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(groupID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(groupID)) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(groupID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(groupID)) } changed := false @@ -500,13 +501,13 @@ func (m *GroupManager) UpdateKeyspaceGroup(oldGroupID, newGroupID string, oldUse return errors.Errorf("keyspace group %s not found in %s group", newGroupID, newUserKind) } if oldKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(oldID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(oldID)) } else if newKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(newID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(newID)) } else if oldKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(oldID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(oldID)) } else if newKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(newID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(newID)) } var updateOld, updateNew bool @@ -552,15 +553,15 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitSourceID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitSourceID) } // A keyspace group can not take part in multiple split processes. if splitSourceKg.IsSplitting() { - return ErrKeyspaceGroupInSplit(splitSourceID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(splitSourceID) } // A keyspace group can not be split when it is in merging. if splitSourceKg.IsMerging() { - return ErrKeyspaceGroupInMerging(splitSourceID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(splitSourceID) } // Build the new keyspace groups for split source and target. var startKeyspaceID, endKeyspaceID uint32 @@ -574,7 +575,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( } // Check if the source keyspace group has enough replicas. if len(splitSourceKg.Members) < constant.DefaultKeyspaceGroupReplicaCount { - return ErrKeyspaceGroupNotEnoughReplicas + return errs.ErrKeyspaceGroupNotEnoughReplicas } // Check if the new keyspace group already exists. splitTargetKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetID) @@ -582,7 +583,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitTargetKg != nil { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } // Update the old keyspace group. splitSourceKg.Keyspaces = splitSourceKeyspaces @@ -623,7 +624,7 @@ func buildSplitKeyspaces( // Split according to the new keyspace list. if newNum != 0 { if newNum > oldNum { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( oldKeyspaceMap = make(map[uint32]struct{}, oldNum) @@ -634,10 +635,10 @@ func buildSplitKeyspaces( } for _, keyspace := range new { if keyspace == constant.DefaultKeyspaceID { - return nil, nil, ErrModifyDefaultKeyspace + return nil, nil, errs.ErrModifyDefaultKeyspace } if _, ok := oldKeyspaceMap[keyspace]; !ok { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } newKeyspaceMap[keyspace] = struct{}{} } @@ -662,7 +663,7 @@ func buildSplitKeyspaces( } // Split according to the start and end keyspace ID. if startKeyspaceID == 0 && endKeyspaceID == 0 { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( newSplit = make([]uint32, 0, oldNum) @@ -681,7 +682,7 @@ func buildSplitKeyspaces( } // Check if the new keyspace list is empty. if len(newSplit) == 0 { - return nil, nil, ErrKeyspaceGroupWithEmptyKeyspace + return nil, nil, errs.ErrKeyspaceGroupWithEmptyKeyspace } // Get the split keyspace list for the old keyspace group. oldSplit := make([]uint32, 0, oldNum-len(newSplit)) @@ -705,11 +706,11 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitTargetKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetID) } // Check if it's in the split state. if !splitTargetKg.IsSplitTarget() { - return ErrKeyspaceGroupNotInSplit(splitTargetID) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetID) } // Load the split source keyspace group then. splitSourceKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetKg.SplitSource()) @@ -717,10 +718,10 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetKg.SplitSource()) } if !splitSourceKg.IsSplitSource() { - return ErrKeyspaceGroupNotInSplit(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetKg.SplitSource()) } splitTargetKg.SplitState = nil splitSourceKg.SplitState = nil @@ -765,13 +766,13 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, existMembers map[st return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } for addr := range existMembers { @@ -788,14 +789,14 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, existMembers map[st case <-ticker.C: } if m.GetNodesCount() == 0 { // double check - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } if len(existMembers) == m.GetNodesCount() { break } addr := m.nodesBalancer.Next() if addr == "" { - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } if _, ok := existMembers[addr]; ok { continue @@ -831,13 +832,13 @@ func (m *GroupManager) SetNodesForKeyspaceGroup(id uint32, nodes []string) error return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } members := make([]endpoint.KeyspaceGroupMember, 0, len(nodes)) for _, node := range nodes { @@ -868,13 +869,13 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } inKeyspaceGroup := false members := make([]endpoint.KeyspaceGroupMember, 0, len(kg.Members)) @@ -886,7 +887,7 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior members = append(members, member) } if !inKeyspaceGroup { - return ErrNodeNotInKeyspaceGroup + return errs.ErrNodeNotInKeyspaceGroup } kg.Members = members return m.store.SaveKeyspaceGroup(txn, kg) @@ -920,10 +921,10 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin // - Load and update the target keyspace group. // So we pre-check the number of operations to avoid exceeding the maximum number of etcd transaction. if (mergeListNum+1)*2 > etcdutil.MaxEtcdTxnOps { - return ErrExceedMaxEtcdTxnOps + return errs.ErrExceedMaxEtcdTxnOps } if slice.Contains(mergeList, constant.DefaultKeyspaceGroupID) { - return ErrModifyDefaultKeyspaceGroup + return errs.ErrModifyDefaultKeyspaceGroup } var ( groups = make(map[uint32]*endpoint.KeyspaceGroup, mergeListNum+1) @@ -939,15 +940,15 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin return err } if kg == nil { - return ErrKeyspaceGroupNotExists(kgID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(kgID) } // A keyspace group can not be merged if it's in splitting. if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(kgID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(kgID) } // A keyspace group can not be split when it is in merging. if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(kgID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(kgID) } groups[kgID] = kg } @@ -1013,11 +1014,11 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if mergeTargetKg == nil { - return ErrKeyspaceGroupNotExists(mergeTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(mergeTargetID) } // Check if it's in the merging state. if !mergeTargetKg.IsMergeTarget() { - return ErrKeyspaceGroupNotInMerging(mergeTargetID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(mergeTargetID) } // Make sure all merging keyspace groups are deleted. for _, kgID := range mergeTargetKg.MergeState.MergeList { @@ -1026,7 +1027,7 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if kg != nil { - return ErrKeyspaceGroupNotInMerging(kgID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(kgID) } } mergeList = mergeTargetKg.MergeState.MergeList @@ -1150,7 +1151,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if kg == nil { - return "", ErrKeyspaceGroupNotExists(id) + return "", errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } primaryPath := keypath.LeaderPath(&keypath.MsParam{ @@ -1163,7 +1164,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if !ok { - return "", ErrKeyspaceGroupPrimaryNotFound + return "", errs.ErrKeyspaceGroupPrimaryNotFound } // The format of leader name is address-groupID. contents := strings.Split(leader.GetName(), "-") diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index c615627be71..68461855c6f 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -23,6 +23,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" @@ -256,13 +257,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { re.NoError(err) // split the default keyspace err = suite.kgm.SplitKeyspaceGroupByID(0, 4, []uint32{constant.DefaultKeyspaceID}) - re.ErrorIs(err, ErrModifyDefaultKeyspace) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspace) // split the keyspace group 1 to 4 err = suite.kgm.SplitKeyspaceGroupByID(1, 4, []uint32{444}) - re.ErrorIs(err, ErrKeyspaceGroupNotEnoughReplicas) + re.ErrorIs(err, errs.ErrKeyspaceGroupNotEnoughReplicas) // split the keyspace group 2 to 4 without giving any keyspace err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) // split the keyspace group 2 to 4 err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) re.NoError(err) @@ -283,25 +284,25 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // finish the split of the keyspace group 2 err = suite.kgm.FinishSplitKeyspaceByID(2) - re.ErrorContains(err, ErrKeyspaceGroupNotInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(2).Error()) // finish the split of a non-existing keyspace group err = suite.kgm.FinishSplitKeyspaceByID(5) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // split the in-split keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) // remove the in-split keyspace group kg2, err = suite.kgm.DeleteKeyspaceGroupByID(2) re.Nil(kg2) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) kg4, err = suite.kgm.DeleteKeyspaceGroupByID(4) re.Nil(kg4) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // update the in-split keyspace group err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "2", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "4", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // finish the split of keyspace group 4 err = suite.kgm.FinishSplitKeyspaceByID(4) @@ -321,13 +322,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // split a non-existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(3, 5, nil) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(3).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(3).Error()) // split into an existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{111}) - re.ErrorIs(err, ErrKeyspaceGroupExists) + re.ErrorIs(err, errs.ErrKeyspaceGroupExists) // split with the wrong keyspaces. err = suite.kgm.SplitKeyspaceGroupByID(2, 5, []uint32{111, 222, 444}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) } func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplitRange() { @@ -449,13 +450,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { // merge a non-existing keyspace group err = suite.kgm.MergeKeyspaceGroups(4, []uint32{5}) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // merge with the number of keyspace groups exceeds the limit err = suite.kgm.MergeKeyspaceGroups(1, make([]uint32, etcdutil.MaxEtcdTxnOps/2)) - re.ErrorIs(err, ErrExceedMaxEtcdTxnOps) + re.ErrorIs(err, errs.ErrExceedMaxEtcdTxnOps) // merge the default keyspace group err = suite.kgm.MergeKeyspaceGroups(1, []uint32{constant.DefaultKeyspaceGroupID}) - re.ErrorIs(err, ErrModifyDefaultKeyspaceGroup) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspaceGroup) } func TestBuildSplitKeyspaces(t *testing.T) { @@ -484,7 +485,7 @@ func TestBuildSplitKeyspaces(t *testing.T) { { old: []uint32{1, 2, 3, 4, 5}, new: []uint32{6}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, { old: []uint32{1, 2}, @@ -545,11 +546,11 @@ func TestBuildSplitKeyspaces(t *testing.T) { old: []uint32{1, 2, 3, 4, 5}, startKeyspaceID: 7, endKeyspaceID: 10, - err: ErrKeyspaceGroupWithEmptyKeyspace, + err: errs.ErrKeyspaceGroupWithEmptyKeyspace, }, { old: []uint32{1, 2, 3, 4, 5}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, } for idx, testCase := range testCases { diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 95b3162eed0..e68a8fa60e8 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -38,60 +38,6 @@ const ( ) var ( - // ErrKeyspaceNotFound is used to indicate target keyspace does not exist. - ErrKeyspaceNotFound = errors.New("keyspace does not exist") - // ErrRegionSplitTimeout indices to split region timeout - ErrRegionSplitTimeout = errors.New("region split timeout") - // ErrRegionSplitFailed indices to split region failed - ErrRegionSplitFailed = errors.New("region split failed") - // ErrKeyspaceExists indicates target keyspace already exists. - // It's used when creating a new keyspace. - ErrKeyspaceExists = errors.New("keyspace already exists") - // ErrKeyspaceGroupExists indicates target keyspace group already exists. - ErrKeyspaceGroupExists = errors.New("keyspace group already exists") - // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. - ErrKeyspaceGroupNotExists = func(groupID uint32) error { - return errors.Errorf("keyspace group %v does not exist", groupID) - } - // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. - ErrKeyspaceGroupInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in split state", groupID) - } - // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. - ErrKeyspaceGroupNotInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in split state", groupID) - } - // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. - ErrKeyspaceGroupInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in merging state", groupID) - } - // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. - ErrKeyspaceGroupNotInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in merging state", groupID) - } - // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. - ErrKeyspaceNotInKeyspaceGroup = errors.New("keyspace is not in this keyspace group") - // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. - ErrKeyspaceNotInAnyKeyspaceGroup = errors.New("keyspace is not in any keyspace group") - // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. - ErrNodeNotInKeyspaceGroup = errors.New("the tso node is not in this keyspace group") - // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. - ErrKeyspaceGroupNotEnoughReplicas = errors.New("not enough replicas in the keyspace group") - // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. - ErrKeyspaceGroupWithEmptyKeyspace = errors.New("keyspace group with empty keyspace") - // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. - ErrModifyDefaultKeyspaceGroup = errors.New("default keyspace group cannot be modified") - // ErrNoAvailableNode is used to indicate no available node in the keyspace group. - ErrNoAvailableNode = errors.New("no available node") - // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. - ErrExceedMaxEtcdTxnOps = errors.New("exceed max etcd txn operations") - // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. - ErrModifyDefaultKeyspace = errors.New("cannot modify default keyspace's state") - errIllegalOperation = errors.New("unknown operation") - - // ErrUnsupportedOperationInKeyspace is used to indicate this is an unsupported operation. - ErrUnsupportedOperationInKeyspace = errors.New("it's a unsupported operation") - // stateTransitionTable lists all allowed next state for the given current state. // Note that transit from any state to itself is allowed for idempotence. stateTransitionTable = map[keyspacepb.KeyspaceState][]keyspacepb.KeyspaceState{ @@ -102,9 +48,6 @@ var ( } // Only keyspaces in the state specified by allowChangeConfig are allowed to change their config. allowChangeConfig = []keyspacepb.KeyspaceState{keyspacepb.KeyspaceState_ENABLED, keyspacepb.KeyspaceState_DISABLED} - - // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. - ErrKeyspaceGroupPrimaryNotFound = errors.New("primary of keyspace group does not exist") ) // validateID check if keyspace falls within the acceptable range. diff --git a/pkg/ratelimit/runner.go b/pkg/ratelimit/runner.go index 211a4f71be1..2a121d17c00 100644 --- a/pkg/ratelimit/runner.go +++ b/pkg/ratelimit/runner.go @@ -16,7 +16,6 @@ package ratelimit import ( "context" - "errors" "sync" "time" @@ -24,6 +23,8 @@ import ( "go.uber.org/zap" "github.com/pingcap/log" + + "github.com/tikv/pd/pkg/errs" ) // RegionHeartbeatStageName is the name of the stage of the region heartbeat. @@ -58,9 +59,6 @@ type Task struct { retained bool } -// ErrMaxWaitingTasksExceeded is returned when the number of waiting tasks exceeds the maximum. -var ErrMaxWaitingTasksExceeded = errors.New("max waiting tasks exceeded") - type taskID struct { id uint64 name string @@ -217,12 +215,12 @@ func (cr *ConcurrentRunner) RunTask(id uint64, name string, f func(context.Conte maxWait := time.Since(cr.pendingTasks[0].submittedAt) if maxWait > cr.maxPendingDuration { runnerFailedTasks.WithLabelValues(cr.name, task.name).Inc() - return ErrMaxWaitingTasksExceeded + return errs.ErrMaxWaitingTasksExceeded } } if pendingTaskNum > maxPendingTaskNum { runnerFailedTasks.WithLabelValues(cr.name, task.name).Inc() - return ErrMaxWaitingTasksExceeded + return errs.ErrMaxWaitingTasksExceeded } } cr.pendingTasks = append(cr.pendingTasks, task) diff --git a/pkg/schedule/checker/rule_checker.go b/pkg/schedule/checker/rule_checker.go index 28d5988ce1c..7350d92cf58 100644 --- a/pkg/schedule/checker/rule_checker.go +++ b/pkg/schedule/checker/rule_checker.go @@ -16,7 +16,6 @@ package checker import ( "context" - "errors" "math" "math/rand" "time" @@ -42,15 +41,6 @@ import ( const maxPendingListLen = 100000 -var ( - errNoStoreToAdd = errors.New("no store to add peer") - errNoStoreToReplace = errors.New("no store to replace peer") - errPeerCannotBeLeader = errors.New("peer cannot be leader") - errPeerCannotBeWitness = errors.New("peer cannot be witness") - errNoNewLeader = errors.New("no new leader") - errRegionNoLeader = errors.New("region no leader") -) - // RuleChecker fix/improve region by placement rules. type RuleChecker struct { PauseController @@ -103,7 +93,7 @@ func (c *RuleChecker) CheckWithFit(region *core.RegionInfo, fit *placement.Regio // skip no leader region if region.GetLeader() == nil { ruleCheckerRegionNoLeaderCounter.Inc() - log.Debug("fail to check region", zap.Uint64("region-id", region.GetID()), zap.Error(errRegionNoLeader)) + log.Debug("fail to check region", zap.Uint64("region-id", region.GetID()), errs.ZapError(errs.ErrRegionNoLeader)) return } @@ -230,7 +220,7 @@ func (c *RuleChecker) addRulePeer(region *core.RegionInfo, fit *placement.Region } } } - return nil, errNoStoreToAdd + return nil, errs.ErrNoStoreToAdd } peer := &metapb.Peer{StoreId: store, Role: rf.Rule.Role.MetaPeerRole(), IsWitness: isWitness} op, err := operator.CreateAddPeerOperator("add-rule-peer", c.cluster, region, peer, operator.OpReplica) @@ -262,7 +252,7 @@ func (c *RuleChecker) replaceUnexpectedRulePeer(region *core.RegionInfo, rf *pla if store == 0 { ruleCheckerNoStoreReplaceCounter.Inc() c.handleFilterState(region, filterByTempState) - return nil, errNoStoreToReplace + return nil, errs.ErrNoStoreToReplace } newPeer := &metapb.Peer{StoreId: store, Role: rf.Rule.Role.MetaPeerRole(), IsWitness: fastFailover} // pick the smallest leader store to avoid the Offline store be snapshot generator bottleneck. @@ -325,7 +315,7 @@ func (c *RuleChecker) fixLooseMatchPeer(region *core.RegionInfo, fit *placement. return operator.CreateTransferLeaderOperator("fix-leader-role", c.cluster, region, peer.GetStoreId(), []uint64{}, 0) } ruleCheckerNotAllowLeaderCounter.Inc() - return nil, errPeerCannotBeLeader + return nil, errs.ErrPeerCannotBeLeader } if region.GetLeader().GetId() == peer.GetId() && rf.Rule.Role == placement.Follower { ruleCheckerFixFollowerRoleCounter.Inc() @@ -335,14 +325,14 @@ func (c *RuleChecker) fixLooseMatchPeer(region *core.RegionInfo, fit *placement. } } ruleCheckerNoNewLeaderCounter.Inc() - return nil, errNoNewLeader + return nil, errs.ErrNoNewLeader } if core.IsVoter(peer) && rf.Rule.Role == placement.Learner { ruleCheckerDemoteVoterRoleCounter.Inc() return operator.CreateDemoteVoterOperator("fix-demote-voter", c.cluster, region, peer) } if region.GetLeader().GetId() == peer.GetId() && rf.Rule.IsWitness { - return nil, errPeerCannotBeWitness + return nil, errs.ErrPeerCannotBeWitness } if !core.IsWitness(peer) && rf.Rule.IsWitness && c.isWitnessEnabled() { c.switchWitnessCache.UpdateTTL(c.cluster.GetCheckerConfig().GetSwitchWitnessInterval()) diff --git a/pkg/schedule/scatter/region_scatterer.go b/pkg/schedule/scatter/region_scatterer.go index fdcbd705398..0e4dd43ecb2 100644 --- a/pkg/schedule/scatter/region_scatterer.go +++ b/pkg/schedule/scatter/region_scatterer.go @@ -55,8 +55,6 @@ var ( scatterUnnecessaryCounter = scatterCounter.WithLabelValues("unnecessary", "") scatterFailCounter = scatterCounter.WithLabelValues("fail", "") scatterSuccessCounter = scatterCounter.WithLabelValues("success", "") - errRegionNotFound = errors.New("region not found") - errEmptyRegion = errors.New("empty region") ) const ( @@ -169,7 +167,7 @@ func (r *RegionScatterer) ScatterRegionsByRange(startKey, endKey []byte, group s regions := r.cluster.ScanRegions(startKey, endKey, -1) if len(regions) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, nil, errEmptyRegion + return 0, nil, errs.ErrEmptyRegion } failures := make(map[uint64]error, len(regions)) regionMap := make(map[uint64]*core.RegionInfo, len(regions)) @@ -188,13 +186,13 @@ func (r *RegionScatterer) ScatterRegionsByRange(startKey, endKey []byte, group s func (r *RegionScatterer) ScatterRegionsByID(regionsID []uint64, group string, retryLimit int, skipStoreLimit bool) (int, map[uint64]error, error) { if len(regionsID) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, nil, errEmptyRegion + return 0, nil, errs.ErrEmptyRegion } if len(regionsID) == 1 { region := r.cluster.GetRegion(regionsID[0]) if region == nil { scatterSkipNoRegionCounter.Inc() - return 0, nil, errRegionNotFound + return 0, nil, errs.ErrRegionNotFound } } failures := make(map[uint64]error, len(regionsID)) @@ -230,7 +228,7 @@ func (r *RegionScatterer) ScatterRegionsByID(regionsID []uint64, group string, r func (r *RegionScatterer) scatterRegions(regions map[uint64]*core.RegionInfo, failures map[uint64]error, group string, retryLimit int, skipStoreLimit bool) (int, error) { if len(regions) < 1 { scatterSkipEmptyRegionCounter.Inc() - return 0, errEmptyRegion + return 0, errs.ErrEmptyRegion } if retryLimit > maxRetryLimit { retryLimit = maxRetryLimit diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index 0245016734c..969113bb17e 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -300,7 +300,7 @@ func UpdateKeyspaceConfig(c *gin.Context) { // Check if the update is supported. for _, mutation := range mutations { if mutation.Key == keyspace.GCManagementType && mutation.Value == keyspace.KeyspaceLevelGC { - err = keyspace.ErrUnsupportedOperationInKeyspace + err = errs.ErrUnsupportedOperationInKeyspace c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return } diff --git a/server/keyspace_service.go b/server/keyspace_service.go index e110742aa45..eeca31bc942 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/tikv/pd/pkg/keyspace" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/keypath" ) @@ -39,9 +39,9 @@ type KeyspaceServer struct { // getErrorHeader returns corresponding ResponseHeader based on err. func getErrorHeader(err error) *pdpb.ResponseHeader { switch err { - case keyspace.ErrKeyspaceExists: + case errs.ErrKeyspaceExists: return wrapErrorToHeader(pdpb.ErrorType_DUPLICATED_ENTRY, err.Error()) - case keyspace.ErrKeyspaceNotFound: + case errs.ErrKeyspaceNotFound: return wrapErrorToHeader(pdpb.ErrorType_ENTRY_NOT_FOUND, err.Error()) default: return wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error()) From ecb31de2bc93e5c191259cbe0a4f558f0290fbb8 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 18 Dec 2024 17:10:24 +0800 Subject: [PATCH 11/33] client: move circuit breaker to `pkg` (#8932) ref tikv/pd#8678, ref tikv/pd#8690 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 2 +- client/inner_client.go | 2 +- client/opt/option.go | 2 +- client/{ => pkg}/circuitbreaker/circuit_breaker.go | 0 client/{ => pkg}/circuitbreaker/circuit_breaker_test.go | 0 errors.toml | 5 ----- pkg/errs/errno.go | 1 - tests/integrations/client/client_test.go | 2 +- 8 files changed, 4 insertions(+), 10 deletions(-) rename client/{ => pkg}/circuitbreaker/circuit_breaker.go (100%) rename client/{ => pkg}/circuitbreaker/circuit_breaker_test.go (100%) diff --git a/client/client.go b/client/client.go index 0e48707cd8d..272d6c597b5 100644 --- a/client/client.go +++ b/client/client.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/metastorage" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/clients/tso" @@ -43,6 +42,7 @@ import ( "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" ) diff --git a/client/inner_client.go b/client/inner_client.go index 045a9a6eed8..91f999dd3b5 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -15,11 +15,11 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/tso" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" sd "github.com/tikv/pd/client/servicediscovery" ) diff --git a/client/opt/option.go b/client/opt/option.go index c7a0bb17195..af95a225fab 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" - cb "github.com/tikv/pd/client/circuitbreaker" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" ) diff --git a/client/circuitbreaker/circuit_breaker.go b/client/pkg/circuitbreaker/circuit_breaker.go similarity index 100% rename from client/circuitbreaker/circuit_breaker.go rename to client/pkg/circuitbreaker/circuit_breaker.go diff --git a/client/circuitbreaker/circuit_breaker_test.go b/client/pkg/circuitbreaker/circuit_breaker_test.go similarity index 100% rename from client/circuitbreaker/circuit_breaker_test.go rename to client/pkg/circuitbreaker/circuit_breaker_test.go diff --git a/errors.toml b/errors.toml index 26b15c38077..785de6662f4 100644 --- a/errors.toml +++ b/errors.toml @@ -996,11 +996,6 @@ error = ''' get allocator manager failed, %s ''' -["PD:tso:ErrGetLocalAllocator"] -error = ''' -get local allocator failed, %s -''' - ["PD:tso:ErrGetMinTS"] error = ''' get min ts failed, %s diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index fd9ff3c95ff..30e24647a3f 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -41,7 +41,6 @@ var ( // tso errors var ( ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) - ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) ErrUpdateTimestamp = errors.Normalize("update timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrUpdateTimestamp")) diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index c0b762d0983..fadfb952e4c 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -45,10 +45,10 @@ import ( "github.com/pingcap/kvproto/pkg/pdpb" pd "github.com/tikv/pd/client" - cb "github.com/tikv/pd/client/circuitbreaker" "github.com/tikv/pd/client/clients/router" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" + cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" sd "github.com/tikv/pd/client/servicediscovery" "github.com/tikv/pd/pkg/core" From c5812ee79fe5090145f6be2d7b3fe73e75d95905 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 19 Dec 2024 11:10:12 +0800 Subject: [PATCH 12/33] pd-ctl: adjust output format for store-limit (#8924) ref tikv/pd#4399 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/schedule/config/store_config.go | 2 +- tools/pd-ctl/pdctl/command/store_command.go | 4 ++-- tools/pd-ctl/tests/store/store_test.go | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/schedule/config/store_config.go b/pkg/schedule/config/store_config.go index 19cca753d1d..5575f0d9d56 100644 --- a/pkg/schedule/config/store_config.go +++ b/pkg/schedule/config/store_config.go @@ -174,7 +174,7 @@ func (c *StoreConfig) CheckRegionSize(size, mergeSize uint64) error { if regionSplitSize == 0 { return nil } - // the smallest of the split regions can not be merge again, so it's size should less merge size. + // the smallest of the split regions can not be merge again, so it's size should be less than merge size. if smallSize := size % regionSplitSize; smallSize <= mergeSize && smallSize != 0 { log.Debug("region size is too small", zap.Uint64("size", size), zap.Uint64("merge-size", mergeSize), zap.Uint64("small-size", smallSize)) return errs.ErrCheckerMergeAgain.FastGenByArgs("the smallest region of the split regions is less than max-merge-region-size, " + diff --git a/tools/pd-ctl/pdctl/command/store_command.go b/tools/pd-ctl/pdctl/command/store_command.go index fc398902508..d48d115346b 100644 --- a/tools/pd-ctl/pdctl/command/store_command.go +++ b/tools/pd-ctl/pdctl/command/store_command.go @@ -566,7 +566,7 @@ func storeLimitCommandFunc(cmd *cobra.Command, args []string) { if args[0] == "all" { prefix = storesLimitPrefix if rate > maxStoreLimit { - cmd.Printf("rate should less than %f for all\n", maxStoreLimit) + cmd.Printf("rate should be less than %.1f for all\n", maxStoreLimit) return } } else { @@ -596,7 +596,7 @@ func storeLimitCommandFunc(cmd *cobra.Command, args []string) { return } if rate > maxStoreLimit { - cmd.Printf("rate should less than %f for all\n", maxStoreLimit) + cmd.Printf("rate should be less than %.1f for all\n", maxStoreLimit) return } postInput["rate"] = rate diff --git a/tools/pd-ctl/tests/store/store_test.go b/tools/pd-ctl/tests/store/store_test.go index 79a83325f8b..8ff36b79abe 100644 --- a/tools/pd-ctl/tests/store/store_test.go +++ b/tools/pd-ctl/tests/store/store_test.go @@ -484,13 +484,13 @@ func TestStore(t *testing.T) { args = []string{"-u", pdAddr, "store", "limit", "all", "201"} output, err = tests.ExecuteCommand(cmd, args...) re.NoError(err) - re.Contains(string(output), "rate should less than") + re.Contains(string(output), "rate should be less than") // store limit all 201 is invalid for label args = []string{"-u", pdAddr, "store", "limit", "all", "engine", "key", "201", "add-peer"} output, err = tests.ExecuteCommand(cmd, args...) re.NoError(err) - re.Contains(string(output), "rate should less than") + re.Contains(string(output), "rate should be less than") } // https://github.com/tikv/pd/issues/5024 From da4b43a68bcc266b6578dbe352437917f8f5fa2b Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 19 Dec 2024 14:00:18 +0800 Subject: [PATCH 13/33] api: support to query whether pd has loaded region (#8749) close tikv/pd#8426, close tikv/pd#8748 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/storage/storage.go | 59 ++++++++++++++----- server/api/health_test.go | 8 +-- server/apiv2/handlers/ready.go | 59 +++++++++++++++++++ server/apiv2/router.go | 1 + tests/server/apiv2/handlers/ready_test.go | 72 +++++++++++++++++++++++ tests/server/apiv2/handlers/testutil.go | 1 + 6 files changed, 181 insertions(+), 19 deletions(-) create mode 100644 server/apiv2/handlers/ready.go create mode 100644 tests/server/apiv2/handlers/ready_test.go diff --git a/pkg/storage/storage.go b/pkg/storage/storage.go index 34f2357c8f7..f83beecd0a1 100644 --- a/pkg/storage/storage.go +++ b/pkg/storage/storage.go @@ -20,6 +20,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/tikv/pd/pkg/core" @@ -75,13 +76,21 @@ func NewRegionStorageWithLevelDBBackend( // TODO: support other KV storage backends like BadgerDB in the future. +type regionSource int + +const ( + unloaded regionSource = iota + fromEtcd + fromLeveldb +) + type coreStorage struct { Storage regionStorage endpoint.RegionStorage - useRegionStorage int32 - regionLoaded bool - mu syncutil.Mutex + useRegionStorage atomic.Bool + regionLoaded regionSource + mu syncutil.RWMutex } // NewCoreStorage creates a new core storage with the given default and region storage. @@ -92,6 +101,7 @@ func NewCoreStorage(defaultStorage Storage, regionStorage endpoint.RegionStorage return &coreStorage{ Storage: defaultStorage, regionStorage: regionStorage, + regionLoaded: unloaded, } } @@ -118,12 +128,12 @@ func TrySwitchRegionStorage(s Storage, useLocalRegionStorage bool) endpoint.Regi if useLocalRegionStorage { // Switch the region storage to regionStorage, all region info will be read/saved by the internal // regionStorage, and in most cases it's LevelDB-backend. - atomic.StoreInt32(&ps.useRegionStorage, 1) + ps.useRegionStorage.Store(true) return ps.regionStorage } // Switch the region storage to defaultStorage, all region info will be read/saved by the internal // defaultStorage, and in most cases it's etcd-backend. - atomic.StoreInt32(&ps.useRegionStorage, 0) + ps.useRegionStorage.Store(false) return ps.Storage } @@ -135,24 +145,29 @@ func TryLoadRegionsOnce(ctx context.Context, s Storage, f func(region *core.Regi return s.LoadRegions(ctx, f) } - if atomic.LoadInt32(&ps.useRegionStorage) == 0 { - return ps.Storage.LoadRegions(ctx, f) - } - ps.mu.Lock() defer ps.mu.Unlock() - if !ps.regionLoaded { + + if !ps.useRegionStorage.Load() { + err := ps.Storage.LoadRegions(ctx, f) + if err == nil { + ps.regionLoaded = fromEtcd + } + return err + } + + if ps.regionLoaded == unloaded { if err := ps.regionStorage.LoadRegions(ctx, f); err != nil { return err } - ps.regionLoaded = true + ps.regionLoaded = fromLeveldb } return nil } // LoadRegion loads one region from storage. func (ps *coreStorage) LoadRegion(regionID uint64, region *metapb.Region) (ok bool, err error) { - if atomic.LoadInt32(&ps.useRegionStorage) > 0 { + if ps.useRegionStorage.Load() { return ps.regionStorage.LoadRegion(regionID, region) } return ps.Storage.LoadRegion(regionID, region) @@ -160,7 +175,7 @@ func (ps *coreStorage) LoadRegion(regionID uint64, region *metapb.Region) (ok bo // LoadRegions loads all regions from storage to RegionsInfo. func (ps *coreStorage) LoadRegions(ctx context.Context, f func(region *core.RegionInfo) []*core.RegionInfo) error { - if atomic.LoadInt32(&ps.useRegionStorage) > 0 { + if ps.useRegionStorage.Load() { return ps.regionStorage.LoadRegions(ctx, f) } return ps.Storage.LoadRegions(ctx, f) @@ -168,7 +183,7 @@ func (ps *coreStorage) LoadRegions(ctx context.Context, f func(region *core.Regi // SaveRegion saves one region to storage. func (ps *coreStorage) SaveRegion(region *metapb.Region) error { - if atomic.LoadInt32(&ps.useRegionStorage) > 0 { + if ps.useRegionStorage.Load() { return ps.regionStorage.SaveRegion(region) } return ps.Storage.SaveRegion(region) @@ -176,7 +191,7 @@ func (ps *coreStorage) SaveRegion(region *metapb.Region) error { // DeleteRegion deletes one region from storage. func (ps *coreStorage) DeleteRegion(region *metapb.Region) error { - if atomic.LoadInt32(&ps.useRegionStorage) > 0 { + if ps.useRegionStorage.Load() { return ps.regionStorage.DeleteRegion(region) } return ps.Storage.DeleteRegion(region) @@ -199,3 +214,17 @@ func (ps *coreStorage) Close() error { } return nil } + +// AreRegionsLoaded returns whether the regions are loaded. +func AreRegionsLoaded(s Storage) bool { + ps := s.(*coreStorage) + ps.mu.RLock() + defer ps.mu.RUnlock() + failpoint.Inject("loadRegionSlow", func() { + failpoint.Return(false) + }) + if ps.useRegionStorage.Load() { + return ps.regionLoaded == fromLeveldb + } + return ps.regionLoaded == fromEtcd +} diff --git a/server/api/health_test.go b/server/api/health_test.go index 40a781a6e60..1720b75b69c 100644 --- a/server/api/health_test.go +++ b/server/api/health_test.go @@ -50,22 +50,22 @@ func TestHealthSlice(t *testing.T) { re := require.New(t) cfgs, svrs, clean := mustNewCluster(re, 3) defer clean() - var leader, follow *server.Server + var leader, follower *server.Server for _, svr := range svrs { if !svr.IsClosed() && svr.GetMember().IsLeader() { leader = svr } else { - follow = svr + follower = svr } } mustBootstrapCluster(re, leader) addr := leader.GetConfig().ClientUrls + apiPrefix + "/api/v1/health" - follow.Close() + follower.Close() resp, err := testDialClient.Get(addr) re.NoError(err) defer resp.Body.Close() buf, err := io.ReadAll(resp.Body) re.NoError(err) - checkSliceResponse(re, buf, cfgs, follow.GetConfig().Name) + checkSliceResponse(re, buf, cfgs, follower.GetConfig().Name) } diff --git a/server/apiv2/handlers/ready.go b/server/apiv2/handlers/ready.go new file mode 100644 index 00000000000..0946bc94eb0 --- /dev/null +++ b/server/apiv2/handlers/ready.go @@ -0,0 +1,59 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package handlers + +import ( + "net/http" + + "github.com/gin-gonic/gin" + + "github.com/tikv/pd/pkg/storage" + "github.com/tikv/pd/server" + "github.com/tikv/pd/server/apiv2/middlewares" +) + +// ReadyStatus reflects the cluster's ready status. +// NOTE: This type is exported by HTTP API. Please pay more attention when modifying it. +type ReadyStatus struct { + RegionLoaded bool `json:"region_loaded"` +} + +// @Summary It will return whether pd follower is ready to became leader. +// @Router /ready [get] +// @Param verbose query bool false "Whether to return details." +// @Success 200 +// @Failure 500 +func Ready(c *gin.Context) { + svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) + s := svr.GetStorage() + regionLoaded := storage.AreRegionsLoaded(s) + if regionLoaded { + c.Status(http.StatusOK) + } else { + c.Status(http.StatusInternalServerError) + } + + if _, ok := c.GetQuery("verbose"); !ok { + return + } + resp := &ReadyStatus{ + RegionLoaded: regionLoaded, + } + if regionLoaded { + c.IndentedJSON(http.StatusOK, resp) + } else { + c.AbortWithStatusJSON(http.StatusInternalServerError, resp) + } +} diff --git a/server/apiv2/router.go b/server/apiv2/router.go index 612ba8932ff..7f927dd9705 100644 --- a/server/apiv2/router.go +++ b/server/apiv2/router.go @@ -53,6 +53,7 @@ func NewV2Handler(_ context.Context, svr *server.Server) (http.Handler, apiutil. }) router.Use(middlewares.Redirector()) root := router.Group(apiV2Prefix) + root.GET("ready", handlers.Ready) handlers.RegisterKeyspace(root) handlers.RegisterTSOKeyspaceGroup(root) handlers.RegisterMicroService(root) diff --git a/tests/server/apiv2/handlers/ready_test.go b/tests/server/apiv2/handlers/ready_test.go new file mode 100644 index 00000000000..111fc06784d --- /dev/null +++ b/tests/server/apiv2/handlers/ready_test.go @@ -0,0 +1,72 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package handlers + +import ( + "context" + "encoding/json" + "io" + "net/http" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/pingcap/failpoint" + + tu "github.com/tikv/pd/pkg/utils/testutil" + "github.com/tikv/pd/server/apiv2/handlers" + "github.com/tikv/pd/tests" +) + +func TestReady(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + cluster, err := tests.NewTestCluster(ctx, 1) + re.NoError(err) + defer cluster.Destroy() + re.NoError(cluster.RunInitialServers()) + re.NotEmpty(cluster.WaitLeader()) + server := cluster.GetLeaderServer() + re.NoError(server.BootstrapCluster()) + url := server.GetConfig().ClientUrls + v2Prefix + "/ready" + failpoint.Enable("github.com/tikv/pd/pkg/storage/loadRegionSlow", `return()`) + checkReady(re, url, false) + failpoint.Disable("github.com/tikv/pd/pkg/storage/loadRegionSlow") + checkReady(re, url, true) +} + +func checkReady(re *require.Assertions, url string, isReady bool) { + expectCode := http.StatusOK + if !isReady { + expectCode = http.StatusInternalServerError + } + resp, err := tests.TestDialClient.Get(url) + re.NoError(err) + defer resp.Body.Close() + buf, err := io.ReadAll(resp.Body) + re.NoError(err) + re.Empty(buf) + re.Equal(expectCode, resp.StatusCode) + r := &handlers.ReadyStatus{} + if isReady { + r.RegionLoaded = true + } + data, err := json.Marshal(r) + re.NoError(err) + err = tu.CheckGetJSON(tests.TestDialClient, url+"?verbose", data, + tu.Status(re, expectCode)) + re.NoError(err) +} diff --git a/tests/server/apiv2/handlers/testutil.go b/tests/server/apiv2/handlers/testutil.go index e5280183c52..2a023338c8b 100644 --- a/tests/server/apiv2/handlers/testutil.go +++ b/tests/server/apiv2/handlers/testutil.go @@ -32,6 +32,7 @@ import ( ) const ( + v2Prefix = "/pd/api/v2" keyspacesPrefix = "/pd/api/v2/keyspaces" keyspaceGroupsPrefix = "/pd/api/v2/tso/keyspace-groups" ) From ac5967544f1b3b076791f34145fd653b90cfbff9 Mon Sep 17 00:00:00 2001 From: tongjian <1045931706@qq.com> Date: Thu, 19 Dec 2024 15:01:00 +0800 Subject: [PATCH 14/33] Metrics: Go runtime metrics (#8927) close tikv/pd#8931 Signed-off-by: bufferflies <1045931706@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/go.mod | 23 +- client/go.sum | 55 +- go.mod | 25 +- go.sum | 69 +- metrics/grafana/pd.json | 1944 ++++++++++++++++++++++++++++++++++++ pkg/basicserver/metrics.go | 8 +- pkg/mcs/utils/util.go | 5 + tests/integrations/go.mod | 23 +- tests/integrations/go.sum | 65 +- tools/go.mod | 23 +- tools/go.sum | 65 +- 11 files changed, 2119 insertions(+), 186 deletions(-) diff --git a/client/go.mod b/client/go.mod index ba7b077f46d..b26abcbea55 100644 --- a/client/go.mod +++ b/client/go.mod @@ -12,8 +12,8 @@ require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 - github.com/prometheus/client_golang v1.18.0 - github.com/stretchr/testify v1.8.2 + github.com/prometheus/client_golang v1.20.5 + github.com/stretchr/testify v1.9.0 go.uber.org/goleak v1.1.11 go.uber.org/zap v1.24.0 golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 @@ -24,22 +24,23 @@ require ( require ( github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/golang/protobuf v1.5.3 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/common v0.46.0 // indirect - github.com/prometheus/procfs v0.12.0 // indirect - github.com/stretchr/objx v0.5.0 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.55.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect + github.com/stretchr/objx v0.5.2 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/net v0.23.0 // indirect - golang.org/x/sys v0.18.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/net v0.26.0 // indirect + golang.org/x/sys v0.22.0 // indirect + golang.org/x/text v0.16.0 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240123012728-ef4313101c80 // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.2 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/client/go.sum b/client/go.sum index e229c47ee96..36c58efb823 100644 --- a/client/go.sum +++ b/client/go.sum @@ -7,8 +7,8 @@ github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= 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/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -26,12 +26,16 @@ github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= @@ -55,28 +59,25 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= -github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= -github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= -github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= -github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= -github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= -github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= +github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/stretchr/objx v0.1.0/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.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= -github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +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/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= @@ -111,8 +112,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -123,13 +124,13 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= +golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= +golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -138,8 +139,8 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.6.0 h1:BOw41kyTf3PuCW1pVQf8+Cyg8pMlkYB1oo9iJ6D/lKM= -golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -153,8 +154,8 @@ google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2N google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= 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.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.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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/go.mod b/go.mod index 381d3579785..9c8a7bc90a5 100644 --- a/go.mod +++ b/go.mod @@ -38,8 +38,8 @@ require ( 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-20241104061623-bce95733dad7 - github.com/prometheus/client_golang v1.19.0 - github.com/prometheus/common v0.51.1 + github.com/prometheus/client_golang v1.20.5 + github.com/prometheus/common v0.55.0 github.com/sasha-s/go-deadlock v0.3.5 github.com/shirou/gopsutil/v3 v3.23.3 github.com/smallnest/chanx v1.2.1-0.20240521153536-01121e21ff99 @@ -89,7 +89,7 @@ require ( 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/cespare/xxhash/v2 v2.3.0 // 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 @@ -138,6 +138,7 @@ require ( github.com/joomcode/errorx v1.0.1 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid/v2 v2.2.7 // indirect github.com/leodido/go-urn v1.4.0 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect @@ -147,6 +148,7 @@ require ( github.com/minio/sio v0.3.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/onsi/gomega v1.20.1 // indirect github.com/pelletier/go-toml/v2 v2.2.2 // indirect @@ -155,8 +157,8 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/client_model v0.6.0 // indirect - github.com/prometheus/procfs v0.13.0 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rs/cors v1.7.0 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect github.com/samber/lo v1.37.0 // indirect @@ -194,20 +196,19 @@ require ( go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 golang.org/x/arch v0.8.0 // indirect - golang.org/x/crypto v0.23.0 // indirect + golang.org/x/crypto v0.24.0 // indirect golang.org/x/image v0.18.0 // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.25.0 // indirect - golang.org/x/oauth2 v0.18.0 // indirect + golang.org/x/net v0.26.0 // indirect + golang.org/x/oauth2 v0.21.0 // indirect golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.20.0 // indirect - golang.org/x/term v0.20.0 // indirect + golang.org/x/sys v0.22.0 // indirect + golang.org/x/term v0.21.0 // indirect golang.org/x/text v0.16.0 // indirect - google.golang.org/appengine v1.6.8 // indirect 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.34.1 // indirect + google.golang.org/protobuf v1.34.2 // 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 0338557d0ab..a2f070397d1 100644 --- a/go.sum +++ b/go.sum @@ -2,9 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.2 h1:ZaGT6LiG7dBzi6zNOvVZwacaXlmf3lRqnC4DQzqyRQw= cloud.google.com/go/compute v1.25.1 h1:ZRpHJedLtTpKgr3RV1Fx23NuaAEN1Zfx9hw1u4aJdjU= -cloud.google.com/go/compute v1.25.1/go.mod h1:oopOIR53ly6viBYxaDhBfJwzUAxf1zE//uf3IB011ls= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= +cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= github.com/AlekSi/gocov-xml v1.0.0 h1:4QctJBgXEkbzeKz6PJy6bt3JSPNSN4I2mITYW+eKUoQ= github.com/AlekSi/gocov-xml v1.0.0/go.mod h1:J0qYeZ6tDg4oZubW9mAAgxlqw39PDfoEkzB3HXSbEuA= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= @@ -78,8 +77,8 @@ github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfV github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= 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/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -217,7 +216,6 @@ github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -305,6 +303,8 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -319,6 +319,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= github.com/leodido/go-urn v1.4.0 h1:WT9HwE9SGECu3lg4d/dIA+jxlljEa1/ffXKmRjqdmIQ= @@ -350,6 +352,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5 h1:BvoENQQU+fZ9uukda/RzCAL/191HHwJA5b13R6diVlY= @@ -409,15 +413,15 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= -github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= -github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= -github.com/prometheus/common v0.51.1 h1:eIjN50Bwglz6a/c3hAgSMcofL3nD+nFQkV6Dd4DsQCw= -github.com/prometheus/common v0.51.1/go.mod h1:lrWtQx+iDfn2mbH5GUzlH9TSHyfZpHkSiG1W7y3sF2Q= -github.com/prometheus/procfs v0.13.0 h1:GqzLlQyfsPbaEHaQkO7tbDlriv/4o5Hudv6OXHGKX7o= -github.com/prometheus/procfs v0.13.0/go.mod h1:cd4PFCR54QLnGKPaKGA6l+cfuNXtht43ZKY6tow0Y1g= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= +github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= @@ -530,7 +534,6 @@ github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/go.mod h1:UETIi67q github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.10 h1:+BqfJTcCzTItrop8mq/lbzL8wSGtj94UO/3U31shqG0= @@ -604,9 +607,8 @@ golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= -golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 h1:QLureRX3moex6NVu/Lr4MGakp9FdA7sBHGBmvRW7NaM= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= @@ -623,7 +625,6 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -646,13 +647,12 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= -golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= -golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= +golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= +golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -660,7 +660,6 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -691,24 +690,20 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y= -golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= +golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.20.0 h1:VnkxpohqXaOBYJtBmEppKUG6mXpi+4O6purfc2+sMhw= -golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= @@ -736,7 +731,6 @@ golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -745,8 +739,6 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= -google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= @@ -769,10 +761,9 @@ google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTp google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= 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.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= -google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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/metrics/grafana/pd.json b/metrics/grafana/pd.json index 8be39af25ee..0f4e91afd50 100644 --- a/metrics/grafana/pd.json +++ b/metrics/grafana/pd.json @@ -13992,6 +13992,1950 @@ ], "title": "DR Autosync", "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 33 + }, + "id": 1611, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "bytes" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 34 + }, + "hiddenSeries": false, + "id": 1612, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "rss-{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "go_memory_classes_heap_objects_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "heap_inuse-{{instance}}--{{job}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "go_memory_classes_heap_stacks_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "stack_inuse-{{instance}}--{{job}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "go_memory_classes_heap_unused_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "heap_unused-{{instance}}--{{job}}", + "refId": "D" + }, + { + "exemplar": true, + "expr": "go_memory_classes_metadata_mcache_free_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}+go_memory_classes_metadata_mcache_inuse_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}+go_memory_classes_metadata_mspan_free_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}+go_memory_classes_metadata_mspan_inuse_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}+go_memory_classes_metadata_other_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "go_runtime_metadata-{{instance}}--{{job}}", + "refId": "E" + }, + { + "exemplar": true, + "expr": "go_memory_classes_os_stacks_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}+go_memory_classes_other_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}+go_memory_classes_profiling_buckets_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "other_memory-{{instance}}--{{job}}", + "refId": "F" + }, + { + "exemplar": true, + "expr": "go_memory_classes_heap_free_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "free_mem_reserved_by_go-{{instance}}--{{job}}", + "refId": "G" + }, + { + "exemplar": true, + "expr": "go_memory_classes_heap_released_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "returned_to_os-{{instance}}--{{job}}", + "refId": "H" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Memory Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "short" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 34 + }, + "hiddenSeries": false, + "id": 1613, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "go_memstats_heap_objects{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "objects--{{instance}}--{{job}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated Live Objects", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "decbytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "percentunit" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 40 + }, + "hiddenSeries": false, + "id": 1614, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "irate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "cpu-usage--{{instance}}--{{job}}", + "refId": "A", + "step": 40 + }, + { + "expr": "(idelta((go_memstats_gc_cpu_fraction{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"} * (go_memstats_last_gc_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"} - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}) * tidb_server_maxprocs{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"})[30s:]) > 0) / 15", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "gc-cpu--{{instance}}--{{job}}", + "refId": "C" + }, + { + "expr": "pd_service_maxprocs{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "quota--{{instance}}--{{job}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CPU Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 40 + }, + "hiddenSeries": false, + "id": 1615, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"0\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "min-{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile!~\"0|1\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "{{quantile}}-{{instance}}--{{job}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"1\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "max-{{instance}}--{{job}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GC STW Duration (last 256 GC cycles)", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "short" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 46 + }, + "hiddenSeries": false, + "id": 1616, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/threads.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": " go_goroutines{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "goroutines-{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "go_threads{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "instant": false, + "interval": "", + "legendFormat": "threads-{{instance}}--{{job}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Goroutine Count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "short" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 46 + }, + "hiddenSeries": false, + "id": 1617, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/alloc-ops.*/", + "yaxis": 2 + }, + { + "alias": "/sweep-ops.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "irate(go_memstats_alloc_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc--{{instance}}--{{job}}", + "refId": "A" + }, + { + "expr": "irate((go_memstats_alloc_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"} - go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"})[30s:])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep--{{instance}}--{{job}}", + "refId": "B" + }, + { + "expr": "irate(go_memstats_mallocs_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc-ops--{{instance}}--{{job}}", + "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep-ops--{{instance}}--{{job}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Allocator Throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "Bps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true + }, + "mappings": [], + "min": 0, + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "bytes" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "free-objects-total" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "unit", + "value": "none" + } + ] + } + ] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 52 + }, + "hiddenSeries": false, + "id": 1618, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "sweep", + "transform": "negative-Y" + }, + { + "alias": "alloc-ops", + "yaxis": 2 + }, + { + "alias": "swepp-ops", + "transform": "negative-Y", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(go_gc_heap_frees_by_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])) by (le,instance,job))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ".999 free-by-size--{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_by_size_bytes_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])/increase(go_gc_heap_frees_by_size_bytes_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "hide": false, + "interval": "", + "legendFormat": "avg free-by-size--{{instance}}--{{job}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "free-bytes-total--{{instance}}--{{job}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_objects_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "free-objects-total--{{instance}}--{{job}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated portion of CPU time", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 52 + }, + "hiddenSeries": false, + "id": 1619, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999,sum(rate(go_sched_pauses_total_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le,instance,job))", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 GC STW--{{instance}}--{{job}}", + "refId": "A", + "step": 40 + }, + { + "exemplar": true, + "expr": "rate(go_sched_pauses_total_gc_seconds_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s]) / rate(go_sched_pauses_total_gc_seconds_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg GC STW--{{instance}}--{{job}}", + "refId": "B", + "step": 40 + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.999,sum(rate(go_sched_pauses_total_other_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (instance,le))", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 non-GC STW--{{instance}}--{{job}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "rate(go_sched_pauses_total_other_seconds_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])/rate(go_sched_pauses_total_other_seconds_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg non-GC STW--{{instance}}--{{job}}", + "refId": "D", + "step": 40 + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.999,sum(rate(go_sched_pauses_stopping_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le,instance,job))", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 GC stopping--{{instance}}--{{job}}", + "refId": "E", + "step": 40 + }, + { + "exemplar": true, + "expr": "rate(go_sched_pauses_stopping_gc_seconds_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s]) / rate(go_sched_pauses_stopping_gc_seconds_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "avg GC stopping--{{instance}}--{{job}}", + "refId": "F", + "step": 40 + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.999,sum(rate(go_sched_pauses_stopping_other_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (instance,le))", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 non-GC stopping--{{instance}}--{{job}}", + "refId": "G" + }, + { + "exemplar": true, + "expr": "rate(go_sched_pauses_stopping_other_seconds_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])/rate(go_sched_pauses_stopping_other_seconds_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": " avg non-GC stopping--{{instance}}--{{job}}", + "refId": "H", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GC STW Latency(>= go1.22.0)", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 58 + }, + "hiddenSeries": false, + "id": 1620, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.999, sum(rate(go_sched_latencies_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le, instance,job))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "P999--{{instance}}--{{job}}", + "refId": "A", + "step": 10 + }, + { + "expr": "histogram_quantile(0.9999, sum(rate(go_sched_latencies_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le, instance,job))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "P9999--{{instance}}--{{job}}", + "refId": "B", + "step": 10 + }, + { + "expr": "histogram_quantile(0.99999, sum(rate(go_sched_latencies_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le, instance,job))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "P99999--{{instance}}--{{job}}", + "refId": "C", + "step": 10 + }, + { + "expr": "histogram_quantile(0.999999, sum(rate(go_sched_latencies_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le, instance,job))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "P999999--{{instance}}--{{job}}", + "refId": "D", + "step": 10 + }, + { + "expr": "histogram_quantile(1, sum(rate(go_sched_latencies_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])) by (le, instance,job))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "max--{{instance}}--{{job}}", + "refId": "E", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Goroutine scheduler latency", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "none" + }, + "overrides": [ + { + "matcher": { + "id": "byFrameRefID", + "options": "D" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 58 + }, + "hiddenSeries": false, + "id": 1621, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "increase(go_gc_cycles_automatic_gc_cycles_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "auto gc-{{instance}}--{{job}}", + "refId": "A", + "step": 40 + }, + { + "exemplar": true, + "expr": "increase(go_gc_cycles_forced_gc_cycles_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "forced gc-{{instance}}--{{job}}", + "refId": "B", + "step": 40 + }, + { + "exemplar": true, + "expr": "increase(go_gc_cycles_total_gc_cycles_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total gc-{{instance}}--{{job}}", + "refId": "C", + "step": 40 + }, + { + "exemplar": true, + "expr": "changes(go_gc_limiter_last_enabled_gc_cycle{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": true, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "gc limiter enabled-{{instance}}--{{job}}", + "refId": "D", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Golang GC", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": {}, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 66 + }, + "hiddenSeries": false, + "id": 1622, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "irate(go_sync_mutex_wait_total_seconds_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[30s])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "mutex_wait_seconds-{{instance}}--{{job}}", + "refId": "A", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sync mutex wait", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": {}, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 66 + }, + "hiddenSeries": false, + "id": 1623, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/GOMEMLIMIT.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "go_gc_gogc_percent{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "GOGC-{{instance}}--{{job}}", + "refId": "A", + "step": 40 + }, + { + "exemplar": true, + "expr": "go_gc_gomemlimit_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "GOMEMLIMIT-{{instance}}--{{job}}", + "refId": "B", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GOGC & GOMEMLIMIT", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "decbytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": {}, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 72 + }, + "hiddenSeries": false, + "id": 1624, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/alloc-objects-total.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(go_gc_heap_allocs_by_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])) by (le,instance,job))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 alloc-by-size--{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_allocs_by_size_bytes_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])/increase(go_gc_heap_allocs_by_size_bytes_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "hide": false, + "interval": "", + "legendFormat": "avg alloc-by-size--{{instance}}--{{job}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_allocs_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "alloc-bytes-total--{{instance}}--{{job}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_allocs_objects_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "alloc-objects-total--{{instance}}--{{job}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Heap alloc", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "decbytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": {}, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 66 + }, + "hiddenSeries": false, + "id": 1625, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false, + "hideEmpty": true, + "hideZero": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/free-objects-total.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.999, sum(rate(go_gc_heap_frees_by_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\",job=~\".*(pd|tso|scheduling).*\"}[1m])) by (le,instance,job))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "P999 free-by-size--{{instance}}--{{job}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_by_size_bytes_sum{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])/increase(go_gc_heap_frees_by_size_bytes_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "hide": false, + "interval": "", + "legendFormat": "avg free-by-size--{{instance}}--{{job}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "free-bytes-total--{{instance}}--{{job}}", + "refId": "C" + }, + { + "exemplar": true, + "expr": "increase(go_gc_heap_frees_objects_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\",job=~\".*(pd|tso|scheduling).*\"}[1m])", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "free-objects-total--{{instance}}--{{job}}", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Heap free", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "decbytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "GO Runtime", + "type": "row" } ], "refresh": "30s", diff --git a/pkg/basicserver/metrics.go b/pkg/basicserver/metrics.go index 4e4ab214ed5..dad79c8763f 100644 --- a/pkg/basicserver/metrics.go +++ b/pkg/basicserver/metrics.go @@ -14,7 +14,10 @@ package server -import "github.com/prometheus/client_golang/prometheus" +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/collectors" +) var ( // ServerMaxProcsGauge records the maxprocs. @@ -46,6 +49,9 @@ var ( ) func init() { + prometheus.DefaultRegisterer.Unregister(collectors.NewGoCollector()) + prometheus.MustRegister(collectors.NewGoCollector(collectors.WithGoCollectorRuntimeMetrics(collectors.MetricsGC, collectors.MetricsMemory, collectors.MetricsScheduler))) + prometheus.MustRegister(ServerMaxProcsGauge) prometheus.MustRegister(ServerMemoryLimit) prometheus.MustRegister(ServerInfoGauge) diff --git a/pkg/mcs/utils/util.go b/pkg/mcs/utils/util.go index 68e7edb3d69..9e187b3a361 100644 --- a/pkg/mcs/utils/util.go +++ b/pkg/mcs/utils/util.go @@ -26,6 +26,7 @@ import ( "github.com/gin-gonic/gin" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/collectors" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" etcdtypes "go.etcd.io/etcd/client/pkg/v3/types" @@ -51,6 +52,10 @@ import ( // PromHandler is a handler to get prometheus metrics. func PromHandler() gin.HandlerFunc { + prometheus.DefaultRegisterer.Unregister(collectors.NewGoCollector()) + if err := prometheus.Register(collectors.NewGoCollector(collectors.WithGoCollectorRuntimeMetrics(collectors.MetricsGC, collectors.MetricsMemory, collectors.MetricsScheduler))); err != nil { + log.Warn("go runtime collectors have already registered", errs.ZapError(err)) + } return func(c *gin.Context) { // register promhttp.HandlerOpts DisableCompression promhttp.InstrumentMetricHandler(prometheus.DefaultRegisterer, promhttp.HandlerFor(prometheus.DefaultGatherer, promhttp.HandlerOpts{ diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 97d637697f0..c5fcb617014 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -16,8 +16,8 @@ require ( github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 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/prometheus/client_golang v1.20.5 + github.com/prometheus/client_model v0.6.1 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 @@ -61,7 +61,7 @@ require ( 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/cespare/xxhash/v2 v2.3.0 // 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 @@ -115,6 +115,7 @@ require ( github.com/joomcode/errorx v1.0.1 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid/v2 v2.2.7 // indirect github.com/leodido/go-urn v1.4.0 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect @@ -124,6 +125,7 @@ require ( github.com/minio/sio v0.3.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // 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.2.2 // indirect @@ -136,8 +138,8 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/common v0.51.1 // indirect - github.com/prometheus/procfs v0.13.0 // indirect + github.com/prometheus/common v0.55.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rs/cors v1.7.0 // indirect github.com/samber/lo v1.37.0 // indirect github.com/sasha-s/go-deadlock v0.3.5 // indirect @@ -184,21 +186,20 @@ require ( go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.8.0 // indirect - golang.org/x/crypto v0.23.0 // indirect + golang.org/x/crypto v0.24.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.18.0 // indirect - golang.org/x/net v0.25.0 // indirect - golang.org/x/oauth2 v0.18.0 // indirect + golang.org/x/net v0.26.0 // indirect + golang.org/x/oauth2 v0.21.0 // indirect golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.20.0 // indirect + golang.org/x/sys v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect - google.golang.org/appengine v1.6.8 // indirect 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.34.1 // indirect + google.golang.org/protobuf v1.34.2 // 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 a14e9a4ab05..5a48549ef65 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -2,9 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.2 h1:ZaGT6LiG7dBzi6zNOvVZwacaXlmf3lRqnC4DQzqyRQw= cloud.google.com/go/compute v1.25.1 h1:ZRpHJedLtTpKgr3RV1Fx23NuaAEN1Zfx9hw1u4aJdjU= -cloud.google.com/go/compute v1.25.1/go.mod h1:oopOIR53ly6viBYxaDhBfJwzUAxf1zE//uf3IB011ls= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= +cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/DATA-DOG/go-sqlmock v1.5.0 h1:Shsta01QNfFxHCfpW6YH2STWB0MudeXXEWMr20OEh60= @@ -76,8 +75,8 @@ github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfV github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= 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/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -211,7 +210,6 @@ github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -295,6 +293,8 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -309,6 +309,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= github.com/leodido/go-urn v1.4.0 h1:WT9HwE9SGECu3lg4d/dIA+jxlljEa1/ffXKmRjqdmIQ= @@ -338,6 +340,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5 h1:BvoENQQU+fZ9uukda/RzCAL/191HHwJA5b13R6diVlY= @@ -402,15 +406,15 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= -github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= -github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= -github.com/prometheus/common v0.51.1 h1:eIjN50Bwglz6a/c3hAgSMcofL3nD+nFQkV6Dd4DsQCw= -github.com/prometheus/common v0.51.1/go.mod h1:lrWtQx+iDfn2mbH5GUzlH9TSHyfZpHkSiG1W7y3sF2Q= -github.com/prometheus/procfs v0.13.0 h1:GqzLlQyfsPbaEHaQkO7tbDlriv/4o5Hudv6OXHGKX7o= -github.com/prometheus/procfs v0.13.0/go.mod h1:cd4PFCR54QLnGKPaKGA6l+cfuNXtht43ZKY6tow0Y1g= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= +github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= @@ -517,7 +521,6 @@ github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/go.mod h1:UETIi67q github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.10 h1:+BqfJTcCzTItrop8mq/lbzL8wSGtj94UO/3U31shqG0= @@ -592,9 +595,8 @@ golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= -golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 h1:QLureRX3moex6NVu/Lr4MGakp9FdA7sBHGBmvRW7NaM= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= @@ -611,7 +613,6 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -634,13 +635,12 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= -golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= -golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= +golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= +golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -648,7 +648,6 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -676,22 +675,18 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y= -golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= +golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= @@ -716,7 +711,6 @@ golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -725,8 +719,6 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= -google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= @@ -751,10 +743,9 @@ google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJai google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= 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.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= -google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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/go.mod b/tools/go.mod index 7a1c7b72f36..ded2e2e82c8 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -24,8 +24,8 @@ require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 - github.com/prometheus/client_golang v1.19.0 - github.com/prometheus/common v0.51.1 + github.com/prometheus/client_golang v1.20.5 + github.com/prometheus/common v0.55.0 github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.9.0 @@ -71,7 +71,7 @@ require ( 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/cespare/xxhash/v2 v2.3.0 // 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 @@ -120,6 +120,7 @@ require ( github.com/joomcode/errorx v1.0.1 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/klauspost/cpuid/v2 v2.2.7 // indirect github.com/leodido/go-urn v1.4.0 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect @@ -129,6 +130,7 @@ require ( github.com/minio/sio v0.3.0 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // 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.2.2 // indirect @@ -141,8 +143,8 @@ require ( github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/client_model v0.6.0 // indirect - github.com/prometheus/procfs v0.13.0 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/rs/cors v1.7.0 // indirect github.com/samber/lo v1.37.0 // indirect github.com/sasha-s/go-deadlock v0.3.5 // indirect @@ -185,19 +187,18 @@ require ( go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/arch v0.8.0 // indirect - golang.org/x/crypto v0.23.0 // indirect + golang.org/x/crypto v0.24.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.18.0 // indirect - golang.org/x/net v0.25.0 // indirect - golang.org/x/oauth2 v0.18.0 // indirect + golang.org/x/net v0.26.0 // indirect + golang.org/x/oauth2 v0.21.0 // indirect golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.20.0 // indirect + golang.org/x/sys v0.22.0 // indirect golang.org/x/time v0.5.0 // indirect - google.golang.org/appengine v1.6.8 // indirect 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.34.1 // indirect + google.golang.org/protobuf v1.34.2 // 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 1bfe598d503..bd68a4f0ca1 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -2,9 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.2 h1:ZaGT6LiG7dBzi6zNOvVZwacaXlmf3lRqnC4DQzqyRQw= cloud.google.com/go/compute v1.25.1 h1:ZRpHJedLtTpKgr3RV1Fx23NuaAEN1Zfx9hw1u4aJdjU= -cloud.google.com/go/compute v1.25.1/go.mod h1:oopOIR53ly6viBYxaDhBfJwzUAxf1zE//uf3IB011ls= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= +cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/KyleBanks/depth v1.2.1 h1:5h8fQADFrWtarTdtDudMmGsC7GPbOAu6RVB3ffsVFHc= @@ -74,8 +73,8 @@ github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfV github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= 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/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= 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= @@ -210,7 +209,6 @@ github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -297,6 +295,8 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.2.7 h1:ZWSB3igEs+d0qvnxR/ZBzXVmxkgt8DdzP6m9pfuVLDM= github.com/klauspost/cpuid/v2 v2.2.7/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= @@ -311,6 +311,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= github.com/leodido/go-urn v1.4.0 h1:WT9HwE9SGECu3lg4d/dIA+jxlljEa1/ffXKmRjqdmIQ= @@ -342,6 +344,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5 h1:BvoENQQU+fZ9uukda/RzCAL/191HHwJA5b13R6diVlY= @@ -405,15 +409,15 @@ github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3g github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= -github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= -github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= -github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= -github.com/prometheus/common v0.51.1 h1:eIjN50Bwglz6a/c3hAgSMcofL3nD+nFQkV6Dd4DsQCw= -github.com/prometheus/common v0.51.1/go.mod h1:lrWtQx+iDfn2mbH5GUzlH9TSHyfZpHkSiG1W7y3sF2Q= -github.com/prometheus/procfs v0.13.0 h1:GqzLlQyfsPbaEHaQkO7tbDlriv/4o5Hudv6OXHGKX7o= -github.com/prometheus/procfs v0.13.0/go.mod h1:cd4PFCR54QLnGKPaKGA6l+cfuNXtht43ZKY6tow0Y1g= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.55.0 h1:KEi6DK7lXW/m7Ig5i47x0vRzuBsHuvJdi5ee6Y3G1dc= +github.com/prometheus/common v0.55.0/go.mod h1:2SECS4xJG1kd8XF9IcM1gMX6510RAEL65zxzNImwdc8= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= @@ -521,7 +525,6 @@ github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/go.mod h1:UETIi67q github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.10 h1:+BqfJTcCzTItrop8mq/lbzL8wSGtj94UO/3U31shqG0= @@ -598,9 +601,8 @@ golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI= -golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 h1:QLureRX3moex6NVu/Lr4MGakp9FdA7sBHGBmvRW7NaM= @@ -618,7 +620,6 @@ golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzB golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -641,13 +642,12 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211123203042-d83791d6bcd9/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac= -golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= -golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= +golang.org/x/oauth2 v0.21.0 h1:tsimM75w1tF/uws5rbeHzIWxEqElMehnc+iW793zsZs= +golang.org/x/oauth2 v0.21.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -655,7 +655,6 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -683,22 +682,18 @@ golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y= -golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= +golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= @@ -724,7 +719,6 @@ golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo= -golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -736,8 +730,6 @@ gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJ gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= -google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= @@ -762,10 +754,9 @@ google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJai google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= 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.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= -google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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= From 7889b677fd1acb67553752660b698f4e2aa6108b Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 19 Dec 2024 18:40:06 +0800 Subject: [PATCH 15/33] scheduler: fix panic in grant hot region scheduler (#8934) close tikv/pd#8933 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/schedule/schedulers/grant_hot_region.go | 4 ++++ pkg/schedule/schedulers/hot_region.go | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 837d7290ff7..79be126f1d4 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -320,6 +320,10 @@ func (s *grantHotRegionScheduler) transfer(cluster sche.SchedulerCluster, region } else { op, err = operator.CreateMovePeerOperator(s.GetName()+"-move", cluster, srcRegion, operator.OpRegion|operator.OpLeader, srcStore.GetID(), dstStore) } + if err != nil { + log.Debug("fail to create grant hot leader operator", errs.ZapError(err)) + return + } op.SetPriorityLevel(constant.High) return } diff --git a/pkg/schedule/schedulers/hot_region.go b/pkg/schedule/schedulers/hot_region.go index 622e617b729..2ba9af782db 100644 --- a/pkg/schedule/schedulers/hot_region.go +++ b/pkg/schedule/schedulers/hot_region.go @@ -1352,7 +1352,7 @@ func (bs *balanceSolver) splitBucketsOperator(region *core.RegionInfo, keys [][] op, err := operator.CreateSplitRegionOperator(desc, region, operator.OpSplit, pdpb.CheckPolicy_USEKEY, splitKeys) if err != nil { - log.Error("fail to create split operator", + log.Debug("fail to create split operator", zap.Stringer("resource-type", bs.resourceTy), errs.ZapError(err)) return nil From 0e5d49fc2d63084e4ebdec7a951996ed4b756c70 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Fri, 20 Dec 2024 11:32:27 +0800 Subject: [PATCH 16/33] mcs: fix rule manager initialize (#8937) close tikv/pd#8935 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/mcs/scheduling/server/cluster.go | 2 +- pkg/mock/mockcluster/mockcluster.go | 2 +- pkg/schedule/placement/rule_manager.go | 12 +++- pkg/schedule/placement/rule_manager_test.go | 6 +- pkg/statistics/region_collection_test.go | 6 +- server/cluster/cluster.go | 2 +- server/cluster/cluster_test.go | 10 ++-- server/server.go | 2 +- .../integrations/mcs/scheduling/rule_test.go | 60 ++++++++++++++++++- .../mcs/scheduling/server_test.go | 24 ++++---- tests/scheduling_cluster.go | 11 +++- tests/server/api/operator_test.go | 2 +- tests/testutil.go | 2 +- tools/pd-ctl/tests/config/config_test.go | 4 +- 14 files changed, 110 insertions(+), 35 deletions(-) diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 20e5acca379..5c7166fba09 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -113,7 +113,7 @@ func NewCluster( logRunner: ratelimit.NewConcurrentRunner(logTaskRunner, ratelimit.NewConcurrencyLimiter(uint64(runtime.NumCPU()*2)), time.Minute), } c.coordinator = schedule.NewCoordinator(ctx, c, hbStreams) - err = c.ruleManager.Initialize(persistConfig.GetMaxReplicas(), persistConfig.GetLocationLabels(), persistConfig.GetIsolationLevel()) + err = c.ruleManager.Initialize(persistConfig.GetMaxReplicas(), persistConfig.GetLocationLabels(), persistConfig.GetIsolationLevel(), true) if err != nil { cancel() return nil, err diff --git a/pkg/mock/mockcluster/mockcluster.go b/pkg/mock/mockcluster/mockcluster.go index 45d8e35a0bc..62498bff84f 100644 --- a/pkg/mock/mockcluster/mockcluster.go +++ b/pkg/mock/mockcluster/mockcluster.go @@ -205,7 +205,7 @@ func (mc *Cluster) AllocPeer(storeID uint64) (*metapb.Peer, error) { func (mc *Cluster) initRuleManager() { if mc.RuleManager == nil { mc.RuleManager = placement.NewRuleManager(mc.ctx, mc.GetStorage(), mc, mc.GetSharedConfig()) - mc.RuleManager.Initialize(int(mc.GetReplicationConfig().MaxReplicas), mc.GetReplicationConfig().LocationLabels, mc.GetReplicationConfig().IsolationLevel) + mc.RuleManager.Initialize(int(mc.GetReplicationConfig().MaxReplicas), mc.GetReplicationConfig().LocationLabels, mc.GetReplicationConfig().IsolationLevel, false) } } diff --git a/pkg/schedule/placement/rule_manager.go b/pkg/schedule/placement/rule_manager.go index f5101f0250c..4470ff28424 100644 --- a/pkg/schedule/placement/rule_manager.go +++ b/pkg/schedule/placement/rule_manager.go @@ -35,6 +35,7 @@ import ( "github.com/tikv/pd/pkg/core/constant" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/schedule/config" + "github.com/tikv/pd/pkg/schedule/rangelist" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/storage/kv" @@ -81,12 +82,21 @@ func NewRuleManager(ctx context.Context, storage endpoint.RuleStorage, storeSetI // Initialize loads rules from storage. If Placement Rules feature is never enabled, it creates default rule that is // compatible with previous configuration. -func (m *RuleManager) Initialize(maxReplica int, locationLabels []string, isolationLevel string) error { +func (m *RuleManager) Initialize(maxReplica int, locationLabels []string, isolationLevel string, skipLoadRules bool) error { m.Lock() defer m.Unlock() if m.initialized { return nil } + // If RuleManager is initialized in micro service, + // it will load from etcd watcher and do not modify rule directly. + if skipLoadRules { + m.ruleList = ruleList{ + rangeList: rangelist.List{}, + } + m.initialized = true + return nil + } if err := m.loadRules(); err != nil { return err diff --git a/pkg/schedule/placement/rule_manager_test.go b/pkg/schedule/placement/rule_manager_test.go index 5f5f457da13..29d862ddb5e 100644 --- a/pkg/schedule/placement/rule_manager_test.go +++ b/pkg/schedule/placement/rule_manager_test.go @@ -37,7 +37,7 @@ func newTestManager(t *testing.T, enableWitness bool) (endpoint.RuleStorage, *Ru var err error manager := NewRuleManager(context.Background(), store, nil, mockconfig.NewTestOptions()) manager.conf.SetEnableWitness(enableWitness) - err = manager.Initialize(3, []string{"zone", "rack", "host"}, "") + err = manager.Initialize(3, []string{"zone", "rack", "host"}, "", false) re.NoError(err) return store, manager } @@ -160,7 +160,7 @@ func TestSaveLoad(t *testing.T) { } m2 := NewRuleManager(context.Background(), store, nil, nil) - err := m2.Initialize(3, []string{"no", "labels"}, "") + err := m2.Initialize(3, []string{"no", "labels"}, "", false) re.NoError(err) re.Len(m2.GetAllRules(), 3) re.Equal(m2.GetRule(DefaultGroupID, DefaultRuleID).String(), rules[0].String()) @@ -178,7 +178,7 @@ func TestSetAfterGet(t *testing.T) { manager.SetRule(rule) m2 := NewRuleManager(context.Background(), store, nil, nil) - err := m2.Initialize(100, []string{}, "") + err := m2.Initialize(100, []string{}, "", false) re.NoError(err) rule = m2.GetRule(DefaultGroupID, DefaultRuleID) re.Equal(1, rule.Count) diff --git a/pkg/statistics/region_collection_test.go b/pkg/statistics/region_collection_test.go index f97dffa893d..64442423c02 100644 --- a/pkg/statistics/region_collection_test.go +++ b/pkg/statistics/region_collection_test.go @@ -33,7 +33,7 @@ func TestRegionStatistics(t *testing.T) { re := require.New(t) store := storage.NewStorageWithMemoryBackend() manager := placement.NewRuleManager(context.Background(), store, nil, nil) - err := manager.Initialize(3, []string{"zone", "rack", "host"}, "") + err := manager.Initialize(3, []string{"zone", "rack", "host"}, "", false) re.NoError(err) opt := mockconfig.NewTestOptions() opt.SetPlacementRuleEnabled(false) @@ -122,7 +122,7 @@ func TestRegionStatisticsWithPlacementRule(t *testing.T) { re := require.New(t) store := storage.NewStorageWithMemoryBackend() manager := placement.NewRuleManager(context.Background(), store, nil, nil) - err := manager.Initialize(3, []string{"zone", "rack", "host"}, "") + err := manager.Initialize(3, []string{"zone", "rack", "host"}, "", false) re.NoError(err) opt := mockconfig.NewTestOptions() opt.SetPlacementRuleEnabled(true) @@ -276,7 +276,7 @@ func BenchmarkObserve(b *testing.B) { // Setup store := storage.NewStorageWithMemoryBackend() manager := placement.NewRuleManager(context.Background(), store, nil, nil) - manager.Initialize(3, []string{"zone", "rack", "host"}, "") + manager.Initialize(3, []string{"zone", "rack", "host"}, "", false) opt := mockconfig.NewTestOptions() opt.SetPlacementRuleEnabled(false) peers := []*metapb.Peer{ diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 5ecd787956d..699b43e7901 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -307,7 +307,7 @@ func (c *RaftCluster) InitCluster( c.hbstreams = hbstreams c.ruleManager = placement.NewRuleManager(c.ctx, c.storage, c, c.GetOpts()) if c.opt.IsPlacementRulesEnabled() { - err := c.ruleManager.Initialize(c.opt.GetMaxReplicas(), c.opt.GetLocationLabels(), c.opt.GetIsolationLevel()) + err := c.ruleManager.Initialize(c.opt.GetMaxReplicas(), c.opt.GetLocationLabels(), c.opt.GetIsolationLevel(), false) if err != nil { return err } diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index 11b5743fa13..2f6d04bbf52 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -253,7 +253,7 @@ func TestSetOfflineStore(t *testing.T) { cluster.coordinator = schedule.NewCoordinator(ctx, cluster, nil) cluster.ruleManager = placement.NewRuleManager(ctx, storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts()) if opt.IsPlacementRulesEnabled() { - err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel()) + err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel(), false) if err != nil { panic(err) } @@ -450,7 +450,7 @@ func TestUpStore(t *testing.T) { cluster.coordinator = schedule.NewCoordinator(ctx, cluster, nil) cluster.ruleManager = placement.NewRuleManager(ctx, storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts()) if opt.IsPlacementRulesEnabled() { - err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel()) + err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel(), false) if err != nil { panic(err) } @@ -553,7 +553,7 @@ func TestDeleteStoreUpdatesClusterVersion(t *testing.T) { cluster.coordinator = schedule.NewCoordinator(ctx, cluster, nil) cluster.ruleManager = placement.NewRuleManager(ctx, storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts()) if opt.IsPlacementRulesEnabled() { - err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel()) + err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel(), false) if err != nil { panic(err) } @@ -1324,7 +1324,7 @@ func TestOfflineAndMerge(t *testing.T) { cluster.coordinator = schedule.NewCoordinator(ctx, cluster, nil) cluster.ruleManager = placement.NewRuleManager(ctx, storage.NewStorageWithMemoryBackend(), cluster, cluster.GetOpts()) if opt.IsPlacementRulesEnabled() { - err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel()) + err := cluster.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel(), false) if err != nil { panic(err) } @@ -2187,7 +2187,7 @@ func newTestRaftCluster( rc.InitCluster(id, opt, nil, nil) rc.ruleManager = placement.NewRuleManager(ctx, storage.NewStorageWithMemoryBackend(), rc, opt) if opt.IsPlacementRulesEnabled() { - err := rc.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel()) + err := rc.ruleManager.Initialize(opt.GetMaxReplicas(), opt.GetLocationLabels(), opt.GetIsolationLevel(), false) if err != nil { panic(err) } diff --git a/server/server.go b/server/server.go index a2cc32db9dd..3f397da4d0b 100644 --- a/server/server.go +++ b/server/server.go @@ -1047,7 +1047,7 @@ func (s *Server) SetReplicationConfig(cfg sc.ReplicationConfig) error { } if cfg.EnablePlacementRules { // initialize rule manager. - if err := rc.GetRuleManager().Initialize(int(cfg.MaxReplicas), cfg.LocationLabels, cfg.IsolationLevel); err != nil { + if err := rc.GetRuleManager().Initialize(int(cfg.MaxReplicas), cfg.LocationLabels, cfg.IsolationLevel, false); err != nil { return err } } else { diff --git a/tests/integrations/mcs/scheduling/rule_test.go b/tests/integrations/mcs/scheduling/rule_test.go index a137619afbf..880dfddbb16 100644 --- a/tests/integrations/mcs/scheduling/rule_test.go +++ b/tests/integrations/mcs/scheduling/rule_test.go @@ -16,11 +16,15 @@ package scheduling import ( "context" + "encoding/json" + "fmt" "sort" "testing" "github.com/stretchr/testify/suite" + "github.com/pingcap/failpoint" + "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/placement" "github.com/tikv/pd/pkg/utils/testutil" @@ -46,6 +50,7 @@ func TestRule(t *testing.T) { func (suite *ruleTestSuite) SetupSuite() { re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs", `return(true)`)) var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) @@ -63,12 +68,14 @@ func (suite *ruleTestSuite) SetupSuite() { func (suite *ruleTestSuite) TearDownSuite() { suite.cancel() suite.cluster.Destroy() + re := suite.Require() + re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs")) } func (suite *ruleTestSuite) TestRuleWatch() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoint) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() @@ -205,3 +212,54 @@ func (suite *ruleTestSuite) TestRuleWatch() { re.Equal(labelRule.Labels, labelRules[1].Labels) re.Equal(labelRule.RuleType, labelRules[1].RuleType) } + +func (suite *ruleTestSuite) TestSchedulingSwitch() { + re := suite.Require() + + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 2, suite.cluster) + re.NoError(err) + defer tc.Destroy() + tc.WaitForPrimaryServing(re) + + // Add a new rule from "" to "" + url := fmt.Sprintf("%s/pd/api/v1/config/placement-rule", suite.pdLeaderServer.GetAddr()) + respBundle := make([]placement.GroupBundle, 0) + testutil.Eventually(re, func() bool { + err = testutil.CheckGetJSON(tests.TestDialClient, url, nil, + testutil.StatusOK(re), testutil.ExtractJSON(re, &respBundle)) + re.NoError(err) + return len(respBundle) == 1 && len(respBundle[0].Rules) == 1 + }) + + b2 := placement.GroupBundle{ + ID: "pd", + Index: 1, + Rules: []*placement.Rule{ + {GroupID: "pd", ID: "rule0", Index: 1, Role: placement.Voter, Count: 3}, + }, + } + data, err := json.Marshal(b2) + re.NoError(err) + + err = testutil.CheckPostJSON(tests.TestDialClient, url+"/pd", data, testutil.StatusOK(re)) + re.NoError(err) + testutil.Eventually(re, func() bool { + err = testutil.CheckGetJSON(tests.TestDialClient, url, nil, + testutil.StatusOK(re), testutil.ExtractJSON(re, &respBundle)) + re.NoError(err) + return len(respBundle) == 1 && len(respBundle[0].Rules) == 1 + }) + + // Switch another server + oldPrimary := tc.GetPrimaryServer() + oldPrimary.Close() + tc.WaitForPrimaryServing(re) + newPrimary := tc.GetPrimaryServer() + re.NotEqual(oldPrimary.GetAddr(), newPrimary.GetAddr()) + testutil.Eventually(re, func() bool { + err = testutil.CheckGetJSON(tests.TestDialClient, url, nil, + testutil.StatusOK(re), testutil.ExtractJSON(re, &respBundle)) + re.NoError(err) + return len(respBundle) == 1 && len(respBundle[0].Rules) == 1 + }) +} diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index d3850e4667c..ea1e9df0b50 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -91,7 +91,7 @@ func (suite *serverTestSuite) TearDownSuite() { func (suite *serverTestSuite) TestAllocID() { re := suite.Require() re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/scheduling/server/fastUpdateMember", `return(true)`)) - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -110,7 +110,7 @@ func (suite *serverTestSuite) TestAllocIDAfterLeaderChange() { err = pd2.Run() re.NotEmpty(suite.cluster.WaitLeader()) re.NoError(err) - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -138,7 +138,7 @@ func (suite *serverTestSuite) TestAllocIDAfterLeaderChange() { func (suite *serverTestSuite) TestPrimaryChange() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 2, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 2, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -164,7 +164,7 @@ func (suite *serverTestSuite) TestPrimaryChange() { func (suite *serverTestSuite) TestForwardStoreHeartbeat() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -225,7 +225,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -242,7 +242,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { testutil.Eventually(re, func() bool { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) - tc1, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc1, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc1.Destroy() tc1.WaitForPrimaryServing(re) @@ -278,7 +278,7 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -302,7 +302,7 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { func (suite *serverTestSuite) TestSchedulerSync() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -422,7 +422,7 @@ func checkEvictLeaderStoreIDs(re *require.Assertions, sc *schedulers.Controller, func (suite *serverTestSuite) TestForwardRegionHeartbeat() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -499,7 +499,7 @@ func (suite *serverTestSuite) TestForwardRegionHeartbeat() { func (suite *serverTestSuite) TestStoreLimit() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -660,7 +660,7 @@ func (suite *multipleServerTestSuite) TestReElectLeader() { defer func() { re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/member/skipCampaignLeaderCheck")) }() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) @@ -692,7 +692,7 @@ func (suite *multipleServerTestSuite) TestReElectLeader() { func (suite *serverTestSuite) TestOnlineProgress() { re := suite.Require() - tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.backendEndpoints) + tc, err := tests.NewTestSchedulingCluster(suite.ctx, 1, suite.cluster) re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) diff --git a/tests/scheduling_cluster.go b/tests/scheduling_cluster.go index b5fc2429043..5aa1e220be9 100644 --- a/tests/scheduling_cluster.go +++ b/tests/scheduling_cluster.go @@ -22,6 +22,7 @@ import ( scheduling "github.com/tikv/pd/pkg/mcs/scheduling/server" sc "github.com/tikv/pd/pkg/mcs/scheduling/server/config" + "github.com/tikv/pd/pkg/mcs/utils/constant" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" @@ -31,17 +32,19 @@ import ( type TestSchedulingCluster struct { ctx context.Context + pd *TestCluster backendEndpoints string servers map[string]*scheduling.Server cleanupFuncs map[string]testutil.CleanupFunc } // NewTestSchedulingCluster creates a new scheduling test cluster. -func NewTestSchedulingCluster(ctx context.Context, initialServerCount int, backendEndpoints string) (tc *TestSchedulingCluster, err error) { +func NewTestSchedulingCluster(ctx context.Context, initialServerCount int, pd *TestCluster) (tc *TestSchedulingCluster, err error) { schedulers.Register() tc = &TestSchedulingCluster{ ctx: ctx, - backendEndpoints: backendEndpoints, + pd: pd, + backendEndpoints: pd.GetLeaderServer().GetAddr(), servers: make(map[string]*scheduling.Server, initialServerCount), cleanupFuncs: make(map[string]testutil.CleanupFunc, initialServerCount), } @@ -115,7 +118,9 @@ func (tc *TestSchedulingCluster) WaitForPrimaryServing(re *require.Assertions) * } return false }, testutil.WithWaitFor(10*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - + testutil.Eventually(re, func() bool { + return tc.pd.GetLeaderServer().GetRaftCluster().IsServiceIndependent(constant.SchedulingServiceName) + }) return primary } diff --git a/tests/server/api/operator_test.go b/tests/server/api/operator_test.go index fd08a5ed556..54abdf2d236 100644 --- a/tests/server/api/operator_test.go +++ b/tests/server/api/operator_test.go @@ -472,7 +472,7 @@ func (suite *operatorTestSuite) checkTransferRegionWithPlacementRule(cluster *te svr.GetRaftCluster().GetOpts().GetMaxReplicas(), svr.GetRaftCluster().GetOpts().GetLocationLabels(), svr.GetRaftCluster().GetOpts().GetIsolationLevel(), - ) + false) re.NoError(err) } if len(testCase.rules) > 0 { diff --git a/tests/testutil.go b/tests/testutil.go index 4f2a6beb261..5e99b3dbeda 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -389,7 +389,7 @@ func (s *SchedulingTestEnvironment) startCluster(m SchedulerMode) { re.NoError(leaderServer.BootstrapCluster()) leaderServer.GetRaftCluster().SetPrepared() // start scheduling cluster - tc, err := NewTestSchedulingCluster(ctx, 1, leaderServer.GetAddr()) + tc, err := NewTestSchedulingCluster(ctx, 1, cluster) re.NoError(err) tc.WaitForPrimaryServing(re) tc.GetPrimaryServer().GetCluster().SetPrepared() diff --git a/tools/pd-ctl/tests/config/config_test.go b/tools/pd-ctl/tests/config/config_test.go index cf9e4163457..b6c58fe2bc6 100644 --- a/tools/pd-ctl/tests/config/config_test.go +++ b/tools/pd-ctl/tests/config/config_test.go @@ -740,7 +740,9 @@ func (suite *configTestSuite) checkPlacementRuleBundle(cluster *pdTests.TestClus output, err = tests.ExecuteCommand(cmd, "-u", pdAddr, "config", "placement-rules", "rule-bundle", "get", placement.DefaultGroupID) re.NoError(err) re.NoError(json.Unmarshal(output, &bundle)) - re.Equal(placement.GroupBundle{ID: placement.DefaultGroupID, Index: 0, Override: false, Rules: []*placement.Rule{{GroupID: placement.DefaultGroupID, ID: placement.DefaultRuleID, Role: placement.Voter, Count: 3}}}, bundle) + expect := placement.GroupBundle{ID: placement.DefaultGroupID, Index: 0, Override: false, Rules: []*placement.Rule{{GroupID: placement.DefaultGroupID, ID: placement.DefaultRuleID, Role: placement.Voter, Count: 3}}} + expect.Rules[0].CreateTimestamp = bundle.Rules[0].CreateTimestamp // skip create timestamp in mcs + re.Equal(expect, bundle) f, err := os.CreateTemp("", "pd_tests") re.NoError(err) From 461b86adc78d41355d78fd6b890f5bc9a115f147 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Fri, 20 Dec 2024 13:30:06 +0800 Subject: [PATCH 17/33] client: move `WithCallerComponent` from `RPCClient` to `Client` (#8943) ref tikv/pd#8593 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/client/client.go b/client/client.go index 272d6c597b5..2fc9bd3ef0d 100644 --- a/client/client.go +++ b/client/client.go @@ -102,16 +102,6 @@ type RPCClient interface { // SetExternalTimestamp sets external timestamp SetExternalTimestamp(ctx context.Context, timestamp uint64) error - // WithCallerComponent returns a new RPCClient with the specified caller - // component. Caller component refers to the specific part or module within - // the process. You can set the component in two ways: - // * Define it manually, like `caller.Component("DDL")`. - // * Use the provided helper function, `caller.GetComponent(upperLayer)`. - // The upperLayer parameter specifies the depth of the caller stack, - // where 0 means the current function. Adjust the upperLayer value based - // on your needs. - WithCallerComponent(callerComponent caller.Component) RPCClient - router.Client tso.Client metastorage.Client @@ -138,6 +128,15 @@ type Client interface { // UpdateOption updates the client option. UpdateOption(option opt.DynamicOption, value any) error + // WithCallerComponent returns a new Client with the specified caller + // component. Caller component refers to the specific part or module within + // the process. You can set the component in two ways: + // * Define it manually, like `caller.Component("DDL")`. + // * Use the provided helper function, `caller.GetComponent(upperLayer)`. + // The upperLayer parameter specifies the depth of the caller stack, + // where 0 means the current function. Adjust the upperLayer value based + // on your needs. + WithCallerComponent(callerComponent caller.Component) Client // Close closes the client. Close() @@ -1364,7 +1363,7 @@ func (c *client) respForErr(observer prometheus.Observer, start time.Time, err e } // WithCallerComponent implements the RPCClient interface. -func (c *client) WithCallerComponent(callerComponent caller.Component) RPCClient { +func (c *client) WithCallerComponent(callerComponent caller.Component) Client { newClient := *c newClient.callerComponent = callerComponent return &newClient From 054a3d43e047b2baeb22ad6389c09f337d1fc094 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 25 Dec 2024 15:39:39 +0800 Subject: [PATCH 18/33] *: unify the gRPC errors (#8910) ref tikv/pd#8922 Signed-off-by: Ryan Leung --- errors.toml | 4 +- pkg/errs/errno.go | 67 ++++++++- pkg/mcs/metastorage/server/grpc_service.go | 10 +- .../resourcemanager/server/grpc_service.go | 10 +- pkg/mcs/scheduling/server/grpc_service.go | 10 +- pkg/mcs/tso/server/grpc_service.go | 21 +-- pkg/mcs/tso/server/server.go | 9 +- pkg/syncer/server.go | 4 +- server/api/admin.go | 2 +- server/api/config.go | 2 +- server/forward.go | 10 +- server/grpc_service.go | 138 ++++++++---------- tests/server/cluster/cluster_test.go | 3 +- 13 files changed, 148 insertions(+), 142 deletions(-) diff --git a/errors.toml b/errors.toml index 785de6662f4..2ab3b014f5a 100644 --- a/errors.toml +++ b/errors.toml @@ -661,9 +661,9 @@ error = ''' init file log error, %s ''' -["PD:mcs:ErrNotFoundSchedulingAddr"] +["PD:mcs:ErrNotFoundSchedulingPrimary"] error = ''' -cannot find scheduling address +cannot find scheduling primary ''' ["PD:mcs:ErrSchedulingServer"] diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 30e24647a3f..ee24b4d0673 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -14,7 +14,12 @@ package errs -import "github.com/pingcap/errors" +import ( + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/pingcap/errors" +) const ( // NotLeaderErr indicates the non-leader member received the requests which should be received by leader. @@ -31,6 +36,62 @@ const ( NotServedErr = "is not served" ) +// gRPC errors +var ( + // Canceled indicates the operation was canceled (typically by the caller). + ErrStreamClosed = status.Error(codes.Canceled, "stream is closed") + + // Unknown error. An example of where this error may be returned is + // if a Status value received from another address space belongs to + // an error-space that is not known in this address space. Also + // errors raised by APIs that do not return enough error information + // may be converted to this error. + ErrUnknown = func(err error) error { + return status.Error(codes.Unknown, err.Error()) + } + + // DeadlineExceeded means operation expired before completion. + // For operations that change the state of the system, this error may be + // returned even if the operation has completed successfully. For + // example, a successful response from a server could have been delayed + // long enough for the deadline to expire. + ErrForwardTSOTimeout = status.Error(codes.DeadlineExceeded, "forward tso request timeout") + ErrTSOProxyRecvFromClientTimeout = status.Error(codes.DeadlineExceeded, "tso proxy timeout when receiving from client; stream closed by server") + ErrSendHeartbeatTimeout = status.Error(codes.DeadlineExceeded, "send heartbeat timeout") + + // NotFound means some requested entity (e.g., file or directory) was + // not found. + ErrNotFoundTSOAddr = status.Error(codes.NotFound, "not found tso address") + ErrNotFoundSchedulingAddr = status.Error(codes.NotFound, "not found scheduling address") + ErrNotFoundService = status.Error(codes.NotFound, "not found service") + + // ResourceExhausted indicates some resource has been exhausted, perhaps + // a per-user quota, or perhaps the entire file system is out of space. + ErrMaxCountTSOProxyRoutinesExceeded = status.Error(codes.ResourceExhausted, "max count of concurrent tso proxy routines exceeded") + ErrGRPCRateLimitExceeded = func(err error) error { + return status.Error(codes.ResourceExhausted, err.Error()) + } + + // FailedPrecondition indicates operation was rejected because the + // system is not in a state required for the operation's execution. + // For example, directory to be deleted may be non-empty, an rmdir + // operation is applied to a non-directory, etc. + ErrMismatchClusterID = func(clusterID, requestClusterID uint64) error { + return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", clusterID, requestClusterID) + } + + // Unavailable indicates the service is currently unavailable. + // This is a most likely a transient condition and may be corrected + // by retrying with a backoff. Note that it is not always safe to retry + // non-idempotent operations. + // ErrNotLeader is returned when current server is not the leader and not possible to process request. + // TODO: work as proxy. + ErrNotLeader = status.Error(codes.Unavailable, "not leader") + ErrNotStarted = status.Error(codes.Unavailable, "server not started") + ErrEtcdNotStarted = status.Error(codes.Unavailable, "server is started, but etcd not started") + ErrFollowerHandlingNotAllowed = status.Error(codes.Unavailable, "not leader and follower handling not allowed") +) + // common error in multiple packages var ( ErrGetSourceStore = errors.Normalize("failed to get the source store", errors.RFCCodeText("PD:common:ErrGetSourceStore")) @@ -484,6 +545,6 @@ var ( // Micro service errors var ( - ErrNotFoundSchedulingAddr = errors.Normalize("cannot find scheduling address", errors.RFCCodeText("PD:mcs:ErrNotFoundSchedulingAddr")) - ErrSchedulingServer = errors.Normalize("scheduling server meets %v", errors.RFCCodeText("PD:mcs:ErrSchedulingServer")) + ErrNotFoundSchedulingPrimary = errors.Normalize("cannot find scheduling primary", errors.RFCCodeText("PD:mcs:ErrNotFoundSchedulingPrimary")) + ErrSchedulingServer = errors.Normalize("scheduling server meets %v", errors.RFCCodeText("PD:mcs:ErrSchedulingServer")) ) diff --git a/pkg/mcs/metastorage/server/grpc_service.go b/pkg/mcs/metastorage/server/grpc_service.go index 00f4efb56fd..af09bd3a987 100644 --- a/pkg/mcs/metastorage/server/grpc_service.go +++ b/pkg/mcs/metastorage/server/grpc_service.go @@ -22,23 +22,17 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/log" bs "github.com/tikv/pd/pkg/basicserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" ) -var ( - // errNotLeader is returned when current server is not the leader. - errNotLeader = status.Errorf(codes.Unavailable, "not leader") -) - var _ meta_storagepb.MetaStorageServer = (*Service)(nil) // SetUpRestHandler is a hook to sets up the REST service. @@ -81,7 +75,7 @@ func (*Service) RegisterRESTHandler(_ map[string]http.Handler) error { func (s *Service) checkServing() error { if s.manager == nil || s.manager.srv == nil || !s.manager.srv.IsServing() { - return errNotLeader + return errs.ErrNotLeader } return nil } diff --git a/pkg/mcs/resourcemanager/server/grpc_service.go b/pkg/mcs/resourcemanager/server/grpc_service.go index 6c0d7ce0120..4cc162f7145 100644 --- a/pkg/mcs/resourcemanager/server/grpc_service.go +++ b/pkg/mcs/resourcemanager/server/grpc_service.go @@ -22,8 +22,6 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -31,15 +29,11 @@ import ( "github.com/pingcap/log" bs "github.com/tikv/pd/pkg/basicserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" ) -var ( - // errNotLeader is returned when current server is not the leader. - errNotLeader = status.Errorf(codes.Unavailable, "not leader") -) - var _ rmpb.ResourceManagerServer = (*Service)(nil) // SetUpRestHandler is a hook to sets up the REST service. @@ -89,7 +83,7 @@ func (s *Service) GetManager() *Manager { func (s *Service) checkServing() error { if s.manager == nil || s.manager.srv == nil || !s.manager.srv.IsServing() { - return errNotLeader + return errs.ErrNotLeader } return nil } diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index 440b2d47d4f..3d1183bf734 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -23,8 +23,6 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" @@ -41,12 +39,6 @@ import ( "github.com/tikv/pd/pkg/versioninfo" ) -// gRPC errors -var ( - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrClusterMismatched = status.Errorf(codes.Unavailable, "cluster mismatched") -) - // SetUpRestHandler is a hook to sets up the REST service. var SetUpRestHandler = func(*Service) (http.Handler, apiutil.APIServiceGroup) { return dummyRestService{}, apiutil.APIServiceGroup{} @@ -107,7 +99,7 @@ func (s *heartbeatServer) Send(m core.RegionHeartbeatResponse) error { return errors.WithStack(err) case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return status.Errorf(codes.DeadlineExceeded, "send heartbeat timeout") + return errs.ErrSendHeartbeatTimeout } } diff --git a/pkg/mcs/tso/server/grpc_service.go b/pkg/mcs/tso/server/grpc_service.go index 59abed67213..dc85a730651 100644 --- a/pkg/mcs/tso/server/grpc_service.go +++ b/pkg/mcs/tso/server/grpc_service.go @@ -22,25 +22,18 @@ import ( "time" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" bs "github.com/tikv/pd/pkg/basicserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/keypath" ) -// gRPC errors -var ( - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrClusterMismatched = status.Errorf(codes.Unavailable, "cluster mismatched") -) - var _ tsopb.TSOServer = (*Service)(nil) // SetUpRestHandler is a hook to sets up the REST service. @@ -102,14 +95,12 @@ func (s *Service) Tso(stream tsopb.TSO_TsoServer) error { start := time.Now() // TSO uses leader lease to determine validity. No need to check leader here. if s.IsClosed() { - return status.Errorf(codes.Unknown, "server not started") + return errs.ErrNotStarted } header := request.GetHeader() clusterID := header.GetClusterId() if clusterID != keypath.ClusterID() { - return status.Errorf( - codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", - keypath.ClusterID(), clusterID) + return errs.ErrMismatchClusterID(keypath.ClusterID(), clusterID) } keyspaceID := header.GetKeyspaceId() keyspaceGroupID := header.GetKeyspaceGroupId() @@ -119,7 +110,7 @@ func (s *Service) Tso(stream tsopb.TSO_TsoServer) error { keyspaceID, keyspaceGroupID, count) if err != nil { - return status.Error(codes.Unknown, err.Error()) + return errs.ErrUnknown(err) } keyspaceGroupIDStr := strconv.FormatUint(uint64(keyspaceGroupID), 10) tsoHandleDuration.WithLabelValues(keyspaceGroupIDStr).Observe(time.Since(start).Seconds()) @@ -220,10 +211,10 @@ func (s *Service) GetMinTS( func (s *Service) validRequest(header *tsopb.RequestHeader) (tsopb.ErrorType, error) { if s.IsClosed() || s.keyspaceGroupManager == nil { - return tsopb.ErrorType_NOT_BOOTSTRAPPED, ErrNotStarted + return tsopb.ErrorType_NOT_BOOTSTRAPPED, errs.ErrNotStarted } if header == nil || header.GetClusterId() != keypath.ClusterID() { - return tsopb.ErrorType_CLUSTER_MISMATCHED, ErrClusterMismatched + return tsopb.ErrorType_CLUSTER_MISMATCHED, errs.ErrMismatchClusterID(keypath.ClusterID(), header.GetClusterId()) } return tsopb.ErrorType_OK, nil } diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 34f51573baf..ebd0cca8344 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -30,8 +30,6 @@ import ( "github.com/spf13/cobra" "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -279,7 +277,7 @@ func (s *Server) GetTSOAllocatorManager(keyspaceGroupID uint32) (*tso.AllocatorM // TODO: Check if the sender is from the global TSO allocator func (s *Server) ValidateInternalRequest(_ *tsopb.RequestHeader, _ bool) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrNotStarted } return nil } @@ -288,11 +286,10 @@ func (s *Server) ValidateInternalRequest(_ *tsopb.RequestHeader, _ bool) error { // TODO: Check if the keyspace replica is the primary func (s *Server) ValidateRequest(header *tsopb.RequestHeader) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrNotStarted } if header.GetClusterId() != keypath.ClusterID() { - return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", - keypath.ClusterID(), header.GetClusterId()) + return errs.ErrMismatchClusterID(keypath.ClusterID(), header.GetClusterId()) } return nil } diff --git a/pkg/syncer/server.go b/pkg/syncer/server.go index 89af3f79ccc..150ff738c15 100644 --- a/pkg/syncer/server.go +++ b/pkg/syncer/server.go @@ -23,8 +23,6 @@ import ( "github.com/docker/go-units" "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -208,7 +206,7 @@ func (s *RegionSyncer) Sync(ctx context.Context, stream pdpb.PD_SyncRegionsServe } clusterID := request.GetHeader().GetClusterId() if clusterID != keypath.ClusterID() { - return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", keypath.ClusterID(), clusterID) + return errs.ErrMismatchClusterID(keypath.ClusterID(), clusterID) } log.Info("establish sync region stream", zap.String("requested-server", request.GetMember().GetName()), diff --git a/server/api/admin.go b/server/api/admin.go index 434508c98df..d2be53cf40e 100644 --- a/server/api/admin.go +++ b/server/api/admin.go @@ -231,7 +231,7 @@ func (h *adminHandler) recoverAllocID(w http.ResponseWriter, r *http.Request) { func (h *adminHandler) deleteRegionCacheInSchedulingServer(id ...uint64) error { addr, ok := h.svr.GetServicePrimaryAddr(h.svr.Context(), constant.SchedulingServiceName) if !ok { - return errs.ErrNotFoundSchedulingAddr.FastGenByArgs() + return errs.ErrNotFoundSchedulingPrimary.FastGenByArgs() } var idStr string if len(id) > 0 { diff --git a/server/api/config.go b/server/api/config.go index 3eda889507f..a27a1ed5e9b 100644 --- a/server/api/config.go +++ b/server/api/config.go @@ -566,7 +566,7 @@ func (h *confHandler) GetPDServerConfig(w http.ResponseWriter, _ *http.Request) func (h *confHandler) getSchedulingServerConfig() (*config.Config, error) { addr, ok := h.svr.GetServicePrimaryAddr(h.svr.Context(), constant.SchedulingServiceName) if !ok { - return nil, errs.ErrNotFoundSchedulingAddr.FastGenByArgs() + return nil, errs.ErrNotFoundSchedulingPrimary.FastGenByArgs() } url := fmt.Sprintf("%s/scheduling/api/v1/config", addr) req, err := http.NewRequest(http.MethodGet, url, http.NoBody) diff --git a/server/forward.go b/server/forward.go index b3d0d63c81b..9a604410fc0 100644 --- a/server/forward.go +++ b/server/forward.go @@ -22,8 +22,6 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -107,7 +105,7 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { maxConcurrentTSOProxyStreamings := int32(s.GetMaxConcurrentTSOProxyStreamings()) if maxConcurrentTSOProxyStreamings >= 0 { if newCount := s.concurrentTSOProxyStreamings.Add(1); newCount > maxConcurrentTSOProxyStreamings { - return errors.WithStack(ErrMaxCountTSOProxyRoutinesExceeded) + return errors.WithStack(errs.ErrMaxCountTSOProxyRoutinesExceeded) } } @@ -132,7 +130,7 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { } if request.GetCount() == 0 { err = errs.ErrGenerateTimestamp.FastGenByArgs("tso count should be positive") - return status.Error(codes.Unknown, err.Error()) + return errs.ErrUnknown(err) } forwardCtx, cancelForward, forwardStream, lastForwardedHost, tsoStreamErr, err = s.handleTSOForwarding(forwardCtx, forwardStream, stream, server, request, tsDeadlineCh, lastForwardedHost, cancelForward) if tsoStreamErr != nil { @@ -155,7 +153,7 @@ func (s *GrpcServer) handleTSOForwarding(forwardCtx context.Context, forwardStre ) { forwardedHost, ok := s.GetServicePrimaryAddr(stream.Context(), constant.TSOServiceName) if !ok || len(forwardedHost) == 0 { - return forwardCtx, cancelForward, forwardStream, lastForwardedHost, errors.WithStack(ErrNotFoundTSOAddr), nil + return forwardCtx, cancelForward, forwardStream, lastForwardedHost, errors.WithStack(errs.ErrNotFoundTSOAddr), nil } if forwardStream == nil || lastForwardedHost != forwardedHost { if cancelForward != nil { @@ -458,7 +456,7 @@ func (s *GrpcServer) getGlobalTSO(ctx context.Context) (pdpb.Timestamp, error) { } forwardedHost, ok = s.GetServicePrimaryAddr(ctx, constant.TSOServiceName) if !ok || forwardedHost == "" { - return pdpb.Timestamp{}, ErrNotFoundTSOAddr + return pdpb.Timestamp{}, errs.ErrNotFoundTSOAddr } forwardStream, err = s.getTSOForwardStream(forwardedHost) if err != nil { diff --git a/server/grpc_service.go b/server/grpc_service.go index d10421e87d7..398325cd30a 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -32,8 +32,6 @@ import ( "go.uber.org/multierr" "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -67,23 +65,6 @@ const ( gRPCServiceName = "pdpb.PD" ) -// gRPC errors -var ( - // ErrNotLeader is returned when current server is not the leader and not possible to process request. - // TODO: work as proxy. - ErrNotLeader = status.Errorf(codes.Unavailable, "not leader") - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrSendHeartbeatTimeout = status.Errorf(codes.DeadlineExceeded, "send heartbeat timeout") - ErrNotFoundTSOAddr = status.Errorf(codes.NotFound, "not found tso address") - ErrNotFoundSchedulingAddr = status.Errorf(codes.NotFound, "not found scheduling address") - ErrNotFoundService = status.Errorf(codes.NotFound, "not found service") - ErrForwardTSOTimeout = status.Errorf(codes.DeadlineExceeded, "forward tso request timeout") - ErrMaxCountTSOProxyRoutinesExceeded = status.Errorf(codes.ResourceExhausted, "max count of concurrent tso proxy routines exceeded") - ErrTSOProxyRecvFromClientTimeout = status.Errorf(codes.DeadlineExceeded, "tso proxy timeout when receiving from client; stream closed by server") - ErrEtcdNotStarted = status.Errorf(codes.Unavailable, "server is started, but etcd not started") - ErrFollowerHandlingNotAllowed = status.Errorf(codes.Unavailable, "not leader and follower handling not allowed") -) - var ( errRegionHeartbeatSend = forwardFailCounter.WithLabelValues("region_heartbeat", "send") errRegionHeartbeatClient = forwardFailCounter.WithLabelValues("region_heartbeat", "client") @@ -137,7 +118,7 @@ func (s *tsoServer) send(m *pdpb.TsoResponse) error { return errors.WithStack(err) case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return ErrForwardTSOTimeout + return errs.ErrForwardTSOTimeout } } @@ -167,7 +148,7 @@ func (s *tsoServer) recv(timeout time.Duration) (*pdpb.TsoRequest, error) { return req.request, nil case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return nil, ErrTSOProxyRecvFromClientTimeout + return nil, errs.ErrTSOProxyRecvFromClientTimeout } } @@ -198,7 +179,7 @@ func (s *heartbeatServer) Send(m core.RegionHeartbeatResponse) error { return errors.WithStack(err) case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return ErrSendHeartbeatTimeout + return errs.ErrSendHeartbeatTimeout } } @@ -302,7 +283,7 @@ func (s *GrpcServer) GetMinTS( if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -456,7 +437,7 @@ func (s *GrpcServer) GetMembers(context.Context, *pdpb.GetMembersRequest) (*pdpb if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } // Here we purposely do not check the cluster ID because the client does not know the correct cluster ID @@ -506,7 +487,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return status.Error(codes.ResourceExhausted, err.Error()) + return errs.ErrGRPCRateLimitExceeded(err) } } if s.IsServiceIndependent(constant.TSOServiceName) { @@ -576,7 +557,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { if s.IsServiceIndependent(constant.TSOServiceName) { if request.GetCount() == 0 { err = errs.ErrGenerateTimestamp.FastGenByArgs("tso count should be positive") - return status.Error(codes.Unknown, err.Error()) + return errs.ErrUnknown(err) } forwardCtx, cancelForward, forwardStream, lastForwardedHost, tsoStreamErr, err = s.handleTSOForwarding(forwardCtx, forwardStream, stream, nil, request, tsDeadlineCh, lastForwardedHost, cancelForward) if tsoStreamErr != nil { @@ -591,11 +572,10 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { start := time.Now() // TSO uses leader lease to determine validity. No need to check leader here. if s.IsClosed() { - return status.Errorf(codes.Unknown, "server not started") + return errs.ErrNotStarted } if clusterID := keypath.ClusterID(); request.GetHeader().GetClusterId() != clusterID { - return status.Errorf(codes.FailedPrecondition, - "mismatch cluster id, need %d but got %d", clusterID, request.GetHeader().GetClusterId()) + return errs.ErrMismatchClusterID(clusterID, request.GetHeader().GetClusterId()) } count := request.GetCount() ctx, task := trace.NewTask(ctx, "tso") @@ -603,7 +583,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { task.End() tsoHandleDuration.Observe(time.Since(start).Seconds()) if err != nil { - return status.Error(codes.Unknown, err.Error()) + return errs.ErrUnknown(err) } response := &pdpb.TsoResponse{ Header: wrapHeader(), @@ -624,7 +604,7 @@ func (s *GrpcServer) Bootstrap(ctx context.Context, request *pdpb.BootstrapReque if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -666,7 +646,7 @@ func (s *GrpcServer) IsBootstrapped(ctx context.Context, request *pdpb.IsBootstr if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -693,7 +673,7 @@ func (s *GrpcServer) AllocID(ctx context.Context, request *pdpb.AllocIDRequest) if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -727,7 +707,7 @@ func (s *GrpcServer) IsSnapshotRecovering(ctx context.Context, _ *pdpb.IsSnapsho if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } // recovering mark is stored in etcd directly, there's no need to forward. @@ -751,7 +731,7 @@ func (s *GrpcServer) GetStore(ctx context.Context, request *pdpb.GetStoreRequest if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -805,7 +785,7 @@ func (s *GrpcServer) PutStore(ctx context.Context, request *pdpb.PutStoreRequest if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -860,7 +840,7 @@ func (s *GrpcServer) GetAllStores(ctx context.Context, request *pdpb.GetAllStore if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -903,7 +883,7 @@ func (s *GrpcServer) StoreHeartbeat(ctx context.Context, request *pdpb.StoreHear if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -992,7 +972,7 @@ func (s *GrpcServer) StoreHeartbeat(ctx context.Context, request *pdpb.StoreHear func (s *GrpcServer) updateSchedulingClient(ctx context.Context) (*schedulingClient, error) { forwardedHost, _ := s.GetServicePrimaryAddr(ctx, constant.SchedulingServiceName) if forwardedHost == "" { - return nil, ErrNotFoundSchedulingAddr + return nil, errs.ErrNotFoundSchedulingAddr } pre := s.schedulingClient.Load() @@ -1029,7 +1009,7 @@ type bucketHeartbeatServer struct { func (b *bucketHeartbeatServer) send(bucket *pdpb.ReportBucketsResponse) error { if atomic.LoadInt32(&b.closed) == 1 { - return status.Errorf(codes.Canceled, "stream is closed") + return errs.ErrStreamClosed } done := make(chan error, 1) go func() { @@ -1046,7 +1026,7 @@ func (b *bucketHeartbeatServer) send(bucket *pdpb.ReportBucketsResponse) error { return err case <-timer.C: atomic.StoreInt32(&b.closed, 1) - return ErrSendHeartbeatTimeout + return errs.ErrSendHeartbeatTimeout } } @@ -1082,13 +1062,13 @@ func (s *GrpcServer) ReportBuckets(stream pdpb.PD_ReportBucketsServer) error { if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return status.Error(codes.ResourceExhausted, err.Error()) + return errs.ErrGRPCRateLimitExceeded(err) } } for { request, err := server.recv() failpoint.Inject("grpcClientClosed", func() { - err = status.Error(codes.Canceled, "grpc client closed") + err = errs.ErrStreamClosed request = nil }) if err == io.EOF { @@ -1198,7 +1178,7 @@ func (s *GrpcServer) RegionHeartbeat(stream pdpb.PD_RegionHeartbeatServer) error if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return status.Error(codes.ResourceExhausted, err.Error()) + return errs.ErrGRPCRateLimitExceeded(err) } } for { @@ -1398,7 +1378,7 @@ func (s *GrpcServer) RegionHeartbeat(stream pdpb.PD_RegionHeartbeatServer) error // GetRegion implements gRPC PDServer. func (s *GrpcServer) GetRegion(ctx context.Context, request *pdpb.GetRegionRequest) (*pdpb.GetRegionResponse, error) { failpoint.Inject("rateLimit", func() { - failpoint.Return(nil, status.Error(codes.ResourceExhausted, errs.ErrRateLimitExceeded.Error())) + failpoint.Return(nil, errs.ErrGRPCRateLimitExceeded(errs.ErrRateLimitExceeded)) }) if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { fName := currentFunction() @@ -1406,7 +1386,7 @@ func (s *GrpcServer) GetRegion(ctx context.Context, request *pdpb.GetRegionReque if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1468,7 +1448,7 @@ func (s *GrpcServer) GetPrevRegion(ctx context.Context, request *pdpb.GetRegionR if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1525,7 +1505,7 @@ func (s *GrpcServer) GetRegionByID(ctx context.Context, request *pdpb.GetRegionB if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1585,7 +1565,7 @@ func (s *GrpcServer) ScanRegions(ctx context.Context, request *pdpb.ScanRegionsR if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1641,7 +1621,7 @@ func (s *GrpcServer) BatchScanRegions(ctx context.Context, request *pdpb.BatchSc if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1731,7 +1711,7 @@ func (s *GrpcServer) AskSplit(ctx context.Context, request *pdpb.AskSplitRequest if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1775,7 +1755,7 @@ func (s *GrpcServer) AskBatchSplit(ctx context.Context, request *pdpb.AskBatchSp if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } @@ -1849,7 +1829,7 @@ func (s *GrpcServer) ReportSplit(ctx context.Context, request *pdpb.ReportSplitR if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1885,7 +1865,7 @@ func (s *GrpcServer) ReportBatchSplit(ctx context.Context, request *pdpb.ReportB if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1922,7 +1902,7 @@ func (s *GrpcServer) GetClusterConfig(ctx context.Context, request *pdpb.GetClus if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1952,7 +1932,7 @@ func (s *GrpcServer) PutClusterConfig(ctx context.Context, request *pdpb.PutClus if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -1991,7 +1971,7 @@ func (s *GrpcServer) ScatterRegion(ctx context.Context, request *pdpb.ScatterReg if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } @@ -2103,7 +2083,7 @@ func (s *GrpcServer) GetGCSafePoint(ctx context.Context, request *pdpb.GetGCSafe if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2134,7 +2114,7 @@ func (s *GrpcServer) GetGCSafePoint(ctx context.Context, request *pdpb.GetGCSafe // SyncRegions syncs the regions. func (s *GrpcServer) SyncRegions(stream pdpb.PD_SyncRegionsServer) error { if s.IsClosed() || s.cluster == nil { - return ErrNotStarted + return errs.ErrNotStarted } if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { fName := currentFunction() @@ -2142,12 +2122,12 @@ func (s *GrpcServer) SyncRegions(stream pdpb.PD_SyncRegionsServer) error { if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return status.Error(codes.ResourceExhausted, err.Error()) + return errs.ErrGRPCRateLimitExceeded(err) } } ctx := s.cluster.Context() if ctx == nil { - return ErrNotStarted + return errs.ErrNotStarted } return s.cluster.GetRegionSyncer().Sync(ctx, stream) } @@ -2160,7 +2140,7 @@ func (s *GrpcServer) UpdateGCSafePoint(ctx context.Context, request *pdpb.Update if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2207,7 +2187,7 @@ func (s *GrpcServer) UpdateServiceGCSafePoint(ctx context.Context, request *pdpb if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2261,7 +2241,7 @@ func (s *GrpcServer) GetOperator(ctx context.Context, request *pdpb.GetOperatorR if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } @@ -2334,20 +2314,20 @@ func (s *GrpcServer) validateRequest(header *pdpb.RequestHeader) error { // TODO: Call it in gRPC interceptor. func (s *GrpcServer) validateRoleInRequest(ctx context.Context, header *pdpb.RequestHeader, allowFollower *bool) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrNotStarted } if !s.member.IsLeader() { if allowFollower == nil { - return ErrNotLeader + return errs.ErrNotLeader } if !grpcutil.IsFollowerHandleEnabled(ctx) { // TODO: change the error code - return ErrFollowerHandlingNotAllowed + return errs.ErrFollowerHandlingNotAllowed } *allowFollower = true } if clusterID := keypath.ClusterID(); header.GetClusterId() != clusterID { - return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", clusterID, header.GetClusterId()) + return errs.ErrMismatchClusterID(clusterID, header.GetClusterId()) } return nil } @@ -2473,7 +2453,7 @@ func (s *GrpcServer) SplitRegions(ctx context.Context, request *pdpb.SplitRegion if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } @@ -2537,7 +2517,7 @@ func (s *GrpcServer) SplitAndScatterRegions(ctx context.Context, request *pdpb.S if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2600,7 +2580,7 @@ const globalConfigPath = "/global/config/" // it should be set to `Payload bytes` instead of `Value string` func (s *GrpcServer) StoreGlobalConfig(_ context.Context, request *pdpb.StoreGlobalConfigRequest) (*pdpb.StoreGlobalConfigResponse, error) { if s.client == nil { - return nil, ErrEtcdNotStarted + return nil, errs.ErrEtcdNotStarted } if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { fName := currentFunction() @@ -2608,7 +2588,7 @@ func (s *GrpcServer) StoreGlobalConfig(_ context.Context, request *pdpb.StoreGlo if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } configPath := request.GetConfigPath() @@ -2646,7 +2626,7 @@ func (s *GrpcServer) StoreGlobalConfig(_ context.Context, request *pdpb.StoreGlo // - `ConfigPath` if `Names` is nil can get all values and revision of current path func (s *GrpcServer) LoadGlobalConfig(ctx context.Context, request *pdpb.LoadGlobalConfigRequest) (*pdpb.LoadGlobalConfigResponse, error) { if s.client == nil { - return nil, ErrEtcdNotStarted + return nil, errs.ErrEtcdNotStarted } if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { fName := currentFunction() @@ -2654,7 +2634,7 @@ func (s *GrpcServer) LoadGlobalConfig(ctx context.Context, request *pdpb.LoadGlo if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } configPath := request.GetConfigPath() @@ -2694,7 +2674,7 @@ func (s *GrpcServer) LoadGlobalConfig(ctx context.Context, request *pdpb.LoadGlo // Watch on revision which greater than or equal to the required revision. func (s *GrpcServer) WatchGlobalConfig(req *pdpb.WatchGlobalConfigRequest, server pdpb.PD_WatchGlobalConfigServer) error { if s.client == nil { - return ErrEtcdNotStarted + return errs.ErrEtcdNotStarted } if s.GetServiceMiddlewarePersistOptions().IsGRPCRateLimitEnabled() { fName := currentFunction() @@ -2702,7 +2682,7 @@ func (s *GrpcServer) WatchGlobalConfig(req *pdpb.WatchGlobalConfigRequest, serve if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return status.Error(codes.ResourceExhausted, err.Error()) + return errs.ErrGRPCRateLimitExceeded(err) } } ctx, cancel := context.WithCancel(server.Context()) @@ -2799,7 +2779,7 @@ func (s *GrpcServer) ReportMinResolvedTS(ctx context.Context, request *pdpb.Repo if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2837,7 +2817,7 @@ func (s *GrpcServer) SetExternalTimestamp(ctx context.Context, request *pdpb.Set if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { @@ -2873,7 +2853,7 @@ func (s *GrpcServer) GetExternalTimestamp(ctx context.Context, request *pdpb.Get if done, err := limiter.Allow(fName); err == nil { defer done() } else { - return nil, status.Error(codes.ResourceExhausted, err.Error()) + return nil, errs.ErrGRPCRateLimitExceeded(err) } } fn := func(ctx context.Context, client *grpc.ClientConn) (any, error) { diff --git a/tests/server/cluster/cluster_test.go b/tests/server/cluster/cluster_test.go index b7467eb99a5..357a76ace21 100644 --- a/tests/server/cluster/cluster_test.go +++ b/tests/server/cluster/cluster_test.go @@ -39,6 +39,7 @@ import ( "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/core/storelimit" "github.com/tikv/pd/pkg/dashboard" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mock/mockid" "github.com/tikv/pd/pkg/mock/mockserver" @@ -767,7 +768,7 @@ func TestNotLeader(t *testing.T) { grpcStatus, ok := status.FromError(err) re.True(ok) re.Equal(codes.Unavailable, grpcStatus.Code()) - re.ErrorContains(server.ErrNotLeader, grpcStatus.Message()) + re.ErrorContains(errs.ErrNotLeader, grpcStatus.Message()) } func TestStoreVersionChange(t *testing.T) { From 95bfbe69ae1b48bac2b76b0f229cc030ac89a0ae Mon Sep 17 00:00:00 2001 From: JmPotato Date: Wed, 25 Dec 2024 16:03:03 +0800 Subject: [PATCH 19/33] client/tso: init the ticker when TSO Follower Proxy is already enabled (#8948) close tikv/pd#8947 Init the ticker directly when TSO Follower Proxy is already enabled. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/clients/tso/dispatcher.go | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/client/clients/tso/dispatcher.go b/client/clients/tso/dispatcher.go index bdac8096f85..58722088886 100644 --- a/client/clients/tso/dispatcher.go +++ b/client/clients/tso/dispatcher.go @@ -477,14 +477,22 @@ func (td *tsoDispatcher) connectionCtxsUpdater() { ) log.Info("[tso] start tso connection contexts updater") - setNewUpdateTicker := func(ticker *time.Ticker) { + setNewUpdateTicker := func(interval time.Duration) { if updateTicker.C != nil { updateTicker.Stop() } - updateTicker = ticker + if interval == 0 { + updateTicker = &time.Ticker{} + } else { + updateTicker = time.NewTicker(interval) + } + } + // If the TSO Follower Proxy is enabled, set the update interval to the member update interval. + if option.GetEnableTSOFollowerProxy() { + setNewUpdateTicker(sd.MemberUpdateInterval) } // Set to nil before returning to ensure that the existing ticker can be GC. - defer setNewUpdateTicker(nil) + defer setNewUpdateTicker(0) for { provider.updateConnectionCtxs(ctx, connectionCtxs) @@ -499,13 +507,11 @@ func (td *tsoDispatcher) connectionCtxsUpdater() { if enableTSOFollowerProxy && updateTicker.C == nil { // Because the TSO Follower Proxy is enabled, // the periodic check needs to be performed. - setNewUpdateTicker(time.NewTicker(sd.MemberUpdateInterval)) + setNewUpdateTicker(sd.MemberUpdateInterval) } else if !enableTSOFollowerProxy && updateTicker.C != nil { // Because the TSO Follower Proxy is disabled, // the periodic check needs to be turned off. - setNewUpdateTicker(&time.Ticker{}) - } else { - continue + setNewUpdateTicker(0) } case <-updateTicker.C: // Triggered periodically when the TSO Follower Proxy is enabled. From c2f72acc3335dfaa11fb4b8df0d5cce538db965a Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 25 Dec 2024 17:02:24 +0800 Subject: [PATCH 20/33] api: return not found when region doesn't exist (#8869) close tikv/pd#8868 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/mcs/scheduling/server/apis/v1/api.go | 4 ++++ server/api/region.go | 12 ++++++++++++ server/api/region_test.go | 8 +++++++- tests/integrations/mcs/scheduling/api_test.go | 3 +++ 4 files changed, 26 insertions(+), 1 deletion(-) diff --git a/pkg/mcs/scheduling/server/apis/v1/api.go b/pkg/mcs/scheduling/server/apis/v1/api.go index 3d2d0005a24..535fa79ee0c 100644 --- a/pkg/mcs/scheduling/server/apis/v1/api.go +++ b/pkg/mcs/scheduling/server/apis/v1/api.go @@ -1476,6 +1476,10 @@ func getRegionByID(c *gin.Context) { c.String(http.StatusBadRequest, err.Error()) return } + if regionID == 0 { + c.String(http.StatusBadRequest, errs.ErrRegionInvalidID.FastGenByArgs().Error()) + return + } regionInfo := svr.GetBasicCluster().GetRegion(regionID) if regionInfo == nil { c.String(http.StatusNotFound, errs.ErrRegionNotFound.FastGenByArgs(regionID).Error()) diff --git a/server/api/region.go b/server/api/region.go index afc32d2e762..a439cbfb349 100644 --- a/server/api/region.go +++ b/server/api/region.go @@ -67,8 +67,16 @@ func (h *regionHandler) GetRegionByID(w http.ResponseWriter, r *http.Request) { h.rd.JSON(w, http.StatusBadRequest, err.Error()) return } + if regionID == 0 { + h.rd.JSON(w, http.StatusBadRequest, errs.ErrRegionInvalidID.FastGenByArgs()) + return + } regionInfo := rc.GetRegion(regionID) + if regionInfo == nil { + h.rd.JSON(w, http.StatusNotFound, errs.ErrRegionNotFound.FastGenByArgs(regionID).Error()) + return + } b, err := response.MarshalRegionInfoJSON(r.Context(), regionInfo) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) @@ -101,6 +109,10 @@ func (h *regionHandler) GetRegion(w http.ResponseWriter, r *http.Request) { } regionInfo := rc.GetRegionByKey(paramsByte[0]) + if regionInfo == nil { + h.rd.JSON(w, http.StatusNotFound, errs.ErrRegionNotFound.FastGenByArgs().Error()) + return + } b, err := response.MarshalRegionInfoJSON(r.Context(), regionInfo) if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) diff --git a/server/api/region_test.go b/server/api/region_test.go index ae91b41ef5e..4e0929636e8 100644 --- a/server/api/region_test.go +++ b/server/api/region_test.go @@ -80,7 +80,11 @@ func (suite *regionTestSuite) TestRegion() { r.UpdateBuckets(buckets, r.GetBuckets()) re := suite.Require() mustRegionHeartbeat(re, suite.svr, r) - url := fmt.Sprintf("%s/region/id/%d", suite.urlPrefix, r.GetID()) + url := fmt.Sprintf("%s/region/id/%d", suite.urlPrefix, 0) + re.NoError(tu.CheckGetJSON(testDialClient, url, nil, tu.Status(re, http.StatusBadRequest))) + url = fmt.Sprintf("%s/region/id/%d", suite.urlPrefix, 2333) + re.NoError(tu.CheckGetJSON(testDialClient, url, nil, tu.Status(re, http.StatusNotFound))) + url = fmt.Sprintf("%s/region/id/%d", suite.urlPrefix, r.GetID()) r1 := &response.RegionInfo{} r1m := make(map[string]any) re.NoError(tu.ReadGetJSON(re, testDialClient, url, r1)) @@ -96,6 +100,8 @@ func (suite *regionTestSuite) TestRegion() { re.Equal(core.HexRegionKeyStr([]byte("a")), keys[0].(string)) re.Equal(core.HexRegionKeyStr([]byte("b")), keys[1].(string)) + url = fmt.Sprintf("%s/region/key/%s", suite.urlPrefix, "c") + re.NoError(tu.CheckGetJSON(testDialClient, url, nil, tu.Status(re, http.StatusNotFound))) url = fmt.Sprintf("%s/region/key/%s", suite.urlPrefix, "a") r2 := &response.RegionInfo{} re.NoError(tu.ReadGetJSON(re, testDialClient, url, r2)) diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index 14b867a587d..abace06bb78 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -714,6 +714,9 @@ func (suite *apiTestSuite) checkRegions(cluster *tests.TestCluster) { err = testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &resp) re.NoError(err) re.Equal(3., resp["count"]) + urlPrefix = fmt.Sprintf("%s/scheduling/api/v1/regions/0", scheServerAddr) + testutil.CheckGetJSON(tests.TestDialClient, urlPrefix, nil, + testutil.Status(re, http.StatusBadRequest)) urlPrefix = fmt.Sprintf("%s/scheduling/api/v1/regions/233", scheServerAddr) testutil.CheckGetJSON(tests.TestDialClient, urlPrefix, nil, testutil.Status(re, http.StatusNotFound), testutil.StringContain(re, "not found")) From 4d8009db1b6b5ed9f32255a226dd5608eab36999 Mon Sep 17 00:00:00 2001 From: tongjian <1045931706@qq.com> Date: Thu, 26 Dec 2024 15:42:30 +0800 Subject: [PATCH 21/33] test: add more info for TestDisableSchedulingServiceFallback (#8946) ref tikv/pd#8926 Signed-off-by: bufferflies <1045931706@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- tests/integrations/mcs/scheduling/server_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index ea1e9df0b50..3401fb880cb 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -106,7 +106,7 @@ func (suite *serverTestSuite) TestAllocIDAfterLeaderChange() { re := suite.Require() re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/scheduling/server/fastUpdateMember", `return(true)`)) pd2, err := suite.cluster.Join(suite.ctx) - re.NoError(err) + re.NoError(err, "error: %v", err) err = pd2.Run() re.NotEmpty(suite.cluster.WaitLeader()) re.NoError(err) @@ -261,6 +261,8 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { // API server will execute scheduling jobs since there is no scheduling server. testutil.Eventually(re, func() bool { + re.NotNil(suite.pdLeader.GetServer()) + re.NotNil(suite.pdLeader.GetServer().GetRaftCluster()) return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) leaderServer := suite.pdLeader.GetServer() From 8cd72333f15ffbf391f836f618baa4685ef64a65 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Fri, 27 Dec 2024 16:06:15 +0800 Subject: [PATCH 22/33] client: introduce the connection ctx manager (#8940) ref tikv/pd#8690 Previously, we used a `sync.Map` as a medium to propagate connection ctx updates between the dispatcher and TSO client, which introduced a lot of redundant parameter passing and made the logic less intuitive. This PR implements the same functionality using a common connection ctx manager to simplify and reuse related code. Signed-off-by: JmPotato --- client/clients/tso/client.go | 143 ++++++++------ client/clients/tso/dispatcher.go | 178 +++++------------- client/clients/tso/dispatcher_test.go | 19 +- client/pkg/connectionctx/manager.go | 143 ++++++++++++++ client/pkg/connectionctx/manager_test.go | 83 ++++++++ .../servicediscovery/pd_service_discovery.go | 9 +- 6 files changed, 371 insertions(+), 204 deletions(-) create mode 100644 client/pkg/connectionctx/manager.go create mode 100644 client/pkg/connectionctx/manager_test.go diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index c26dd25f2ad..c6caa8b985f 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -36,6 +36,7 @@ import ( "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" + cctx "github.com/tikv/pd/client/pkg/connectionctx" "github.com/tikv/pd/client/pkg/utils/grpcutil" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" @@ -80,7 +81,9 @@ type Cli struct { svcDiscovery sd.ServiceDiscovery tsoStreamBuilderFactory // leaderURL is the URL of the TSO leader. - leaderURL atomic.Value + leaderURL atomic.Value + conCtxMgr *cctx.Manager[*tsoStream] + updateConCtxsCh chan struct{} // tsoReqPool is the pool to recycle `*tsoRequest`. tsoReqPool *sync.Pool @@ -100,6 +103,8 @@ func NewClient( option: option, svcDiscovery: svcDiscovery, tsoStreamBuilderFactory: factory, + conCtxMgr: cctx.NewManager[*tsoStream](), + updateConCtxsCh: make(chan struct{}, 1), tsoReqPool: &sync.Pool{ New: func() any { return &Request{ @@ -122,6 +127,8 @@ func (c *Cli) getOption() *opt.Option { return c.option } func (c *Cli) getServiceDiscovery() sd.ServiceDiscovery { return c.svcDiscovery } +func (c *Cli) getConnectionCtxMgr() *cctx.Manager[*tsoStream] { return c.conCtxMgr } + func (c *Cli) getDispatcher() *tsoDispatcher { return c.dispatcher.Load() } @@ -133,6 +140,8 @@ func (c *Cli) GetRequestPool() *sync.Pool { // Setup initializes the TSO client. func (c *Cli) Setup() { + // Daemon goroutine to update the connectionCtxs periodically and handle the `connectionCtxs` update event. + go c.connectionCtxsUpdater() if err := c.svcDiscovery.CheckMemberChanged(); err != nil { log.Warn("[tso] failed to check member changed", errs.ZapError(err)) } @@ -154,9 +163,12 @@ func (c *Cli) Close() { log.Info("[tso] tso client is closed") } -// scheduleUpdateTSOConnectionCtxs update the TSO connection contexts. +// scheduleUpdateTSOConnectionCtxs schedules the update of the TSO connection contexts. func (c *Cli) scheduleUpdateTSOConnectionCtxs() { - c.getDispatcher().scheduleUpdateConnectionCtxs() + select { + case c.updateConCtxsCh <- struct{}{}: + default: + } } // GetTSORequest gets a TSO request from the pool. @@ -231,25 +243,66 @@ func (c *Cli) backupClientConn() (*grpc.ClientConn, string) { return nil, "" } -// tsoConnectionContext is used to store the context of a TSO stream connection. -type tsoConnectionContext struct { - ctx context.Context - cancel context.CancelFunc - // Current URL of the stream connection. - streamURL string - // Current stream to send gRPC requests. - stream *tsoStream +// connectionCtxsUpdater updates the `connectionCtxs` regularly. +func (c *Cli) connectionCtxsUpdater() { + log.Info("[tso] start tso connection contexts updater") + + var updateTicker = &time.Ticker{} + setNewUpdateTicker := func(interval time.Duration) { + if updateTicker.C != nil { + updateTicker.Stop() + } + if interval == 0 { + updateTicker = &time.Ticker{} + } else { + updateTicker = time.NewTicker(interval) + } + } + // If the TSO Follower Proxy is enabled, set the update interval to the member update interval. + if c.option.GetEnableTSOFollowerProxy() { + setNewUpdateTicker(sd.MemberUpdateInterval) + } + // Set to nil before returning to ensure that the existing ticker can be GC. + defer setNewUpdateTicker(0) + + ctx, cancel := context.WithCancel(c.ctx) + defer cancel() + for { + c.updateConnectionCtxs(ctx) + select { + case <-ctx.Done(): + log.Info("[tso] exit tso connection contexts updater") + return + case <-c.option.EnableTSOFollowerProxyCh: + enableTSOFollowerProxy := c.option.GetEnableTSOFollowerProxy() + log.Info("[tso] tso follower proxy status changed", + zap.Bool("enable", enableTSOFollowerProxy)) + if enableTSOFollowerProxy && updateTicker.C == nil { + // Because the TSO Follower Proxy is enabled, + // the periodic check needs to be performed. + setNewUpdateTicker(sd.MemberUpdateInterval) + } else if !enableTSOFollowerProxy && updateTicker.C != nil { + // Because the TSO Follower Proxy is disabled, + // the periodic check needs to be turned off. + setNewUpdateTicker(0) + } + case <-updateTicker.C: + // Triggered periodically when the TSO Follower Proxy is enabled. + case <-c.updateConCtxsCh: + // Triggered by the leader/follower change. + } + } } // updateConnectionCtxs will choose the proper way to update the connections. // It will return a bool to indicate whether the update is successful. -func (c *Cli) updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool { +func (c *Cli) updateConnectionCtxs(ctx context.Context) bool { // Normal connection creating, it will be affected by the `enableForwarding`. createTSOConnection := c.tryConnectToTSO if c.option.GetEnableTSOFollowerProxy() { createTSOConnection = c.tryConnectToTSOWithProxy } - if err := createTSOConnection(ctx, connectionCtxs); err != nil { + if err := createTSOConnection(ctx); err != nil { log.Error("[tso] update connection contexts failed", errs.ZapError(err)) return false } @@ -260,30 +313,13 @@ func (c *Cli) updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map // and enableForwarding is true, it will create a new connection to a follower to do the forwarding, // while a new daemon will be created also to switch back to a normal leader connection ASAP the // connection comes back to normal. -func (c *Cli) tryConnectToTSO( - ctx context.Context, - connectionCtxs *sync.Map, -) error { +func (c *Cli) tryConnectToTSO(ctx context.Context) error { var ( - networkErrNum uint64 - err error - stream *tsoStream - url string - cc *grpc.ClientConn - updateAndClear = func(newURL string, connectionCtx *tsoConnectionContext) { - // Only store the `connectionCtx` if it does not exist before. - if connectionCtx != nil { - connectionCtxs.LoadOrStore(newURL, connectionCtx) - } - // Remove all other `connectionCtx`s. - connectionCtxs.Range(func(url, cc any) bool { - if url.(string) != newURL { - cc.(*tsoConnectionContext).cancel() - connectionCtxs.Delete(url) - } - return true - }) - } + networkErrNum uint64 + err error + stream *tsoStream + url string + cc *grpc.ClientConn ) ticker := time.NewTicker(constants.RetryInterval) @@ -292,9 +328,9 @@ func (c *Cli) tryConnectToTSO( for range constants.MaxRetryTimes { c.svcDiscovery.ScheduleCheckMemberChanged() cc, url = c.getTSOLeaderClientConn() - if _, ok := connectionCtxs.Load(url); ok { + if c.conCtxMgr.Exist(url) { // Just trigger the clean up of the stale connection contexts. - updateAndClear(url, nil) + c.conCtxMgr.CleanAllAndStore(ctx, url) return nil } if cc != nil { @@ -305,7 +341,7 @@ func (c *Cli) tryConnectToTSO( err = status.New(codes.Unavailable, "unavailable").Err() }) if stream != nil && err == nil { - updateAndClear(url, &tsoConnectionContext{cctx, cancel, url, stream}) + c.conCtxMgr.CleanAllAndStore(ctx, url, stream) return nil } @@ -348,9 +384,9 @@ func (c *Cli) tryConnectToTSO( forwardedHostTrim := tlsutil.TrimHTTPPrefix(forwardedHost) addr := tlsutil.TrimHTTPPrefix(backupURL) // the goroutine is used to check the network and change back to the original stream - go c.checkLeader(ctx, cancel, forwardedHostTrim, addr, url, updateAndClear) + go c.checkLeader(ctx, cancel, forwardedHostTrim, addr, url) metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addr).Set(1) - updateAndClear(backupURL, &tsoConnectionContext{cctx, cancel, backupURL, stream}) + c.conCtxMgr.CleanAllAndStore(ctx, backupURL, stream) return nil } cancel() @@ -363,7 +399,6 @@ func (c *Cli) checkLeader( ctx context.Context, forwardCancel context.CancelFunc, forwardedHostTrim, addr, url string, - updateAndClear func(newAddr string, connectionCtx *tsoConnectionContext), ) { defer func() { // cancel the forward stream @@ -396,7 +431,7 @@ func (c *Cli) checkLeader( stream, err := c.tsoStreamBuilderFactory.makeBuilder(cc).build(cctx, cancel, c.option.Timeout) if err == nil && stream != nil { log.Info("[tso] recover the original tso stream since the network has become normal", zap.String("url", url)) - updateAndClear(url, &tsoConnectionContext{cctx, cancel, url, stream}) + c.conCtxMgr.CleanAllAndStore(ctx, url, stream) return } } @@ -413,10 +448,7 @@ func (c *Cli) checkLeader( // tryConnectToTSOWithProxy will create multiple streams to all the service endpoints to work as // a TSO proxy to reduce the pressure of the main serving service endpoint. -func (c *Cli) tryConnectToTSOWithProxy( - ctx context.Context, - connectionCtxs *sync.Map, -) error { +func (c *Cli) tryConnectToTSOWithProxy(ctx context.Context) error { tsoStreamBuilders := c.getAllTSOStreamBuilders() leaderAddr := c.svcDiscovery.GetServingURL() forwardedHost := c.getLeaderURL() @@ -424,20 +456,17 @@ func (c *Cli) tryConnectToTSOWithProxy( return errors.Errorf("cannot find the tso leader") } // GC the stale one. - connectionCtxs.Range(func(addr, cc any) bool { - addrStr := addr.(string) - if _, ok := tsoStreamBuilders[addrStr]; !ok { + c.conCtxMgr.GC(func(addr string) bool { + _, ok := tsoStreamBuilders[addr] + if !ok { log.Info("[tso] remove the stale tso stream", - zap.String("addr", addrStr)) - cc.(*tsoConnectionContext).cancel() - connectionCtxs.Delete(addr) + zap.String("addr", addr)) } - return true + return !ok }) // Update the missing one. for addr, tsoStreamBuilder := range tsoStreamBuilders { - _, ok := connectionCtxs.Load(addr) - if ok { + if c.conCtxMgr.Exist(addr) { continue } log.Info("[tso] try to create tso stream", zap.String("addr", addr)) @@ -456,7 +485,7 @@ func (c *Cli) tryConnectToTSOWithProxy( addrTrim := tlsutil.TrimHTTPPrefix(addr) metrics.RequestForwarded.WithLabelValues(forwardedHostTrim, addrTrim).Set(1) } - connectionCtxs.Store(addr, &tsoConnectionContext{cctx, cancel, addr, stream}) + c.conCtxMgr.Store(ctx, addr, stream) continue } log.Error("[tso] create the tso stream failed", diff --git a/client/clients/tso/dispatcher.go b/client/clients/tso/dispatcher.go index 58722088886..c05ab27d755 100644 --- a/client/clients/tso/dispatcher.go +++ b/client/clients/tso/dispatcher.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "math" - "math/rand" "runtime/trace" "sync" "sync/atomic" @@ -36,6 +35,7 @@ import ( "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/batch" + cctx "github.com/tikv/pd/client/pkg/connectionctx" "github.com/tikv/pd/client/pkg/retry" "github.com/tikv/pd/client/pkg/utils/timerutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" @@ -76,7 +76,8 @@ type tsoInfo struct { type tsoServiceProvider interface { getOption() *opt.Option getServiceDiscovery() sd.ServiceDiscovery - updateConnectionCtxs(ctx context.Context, connectionCtxs *sync.Map) bool + getConnectionCtxMgr() *cctx.Manager[*tsoStream] + updateConnectionCtxs(ctx context.Context) bool } const dispatcherCheckRPCConcurrencyInterval = time.Second * 5 @@ -85,12 +86,10 @@ type tsoDispatcher struct { ctx context.Context cancel context.CancelFunc - provider tsoServiceProvider - // URL -> *connectionContext - connectionCtxs *sync.Map - tsoRequestCh chan *Request - tsDeadlineCh chan *deadline - latestTSOInfo atomic.Pointer[tsoInfo] + provider tsoServiceProvider + tsoRequestCh chan *Request + tsDeadlineCh chan *deadline + latestTSOInfo atomic.Pointer[tsoInfo] // For reusing `*batchController` objects batchBufferPool *sync.Pool @@ -102,8 +101,6 @@ type tsoDispatcher struct { lastCheckConcurrencyTime time.Time tokenCount int rpcConcurrency int - - updateConnectionCtxsCh chan struct{} } func newTSODispatcher( @@ -122,12 +119,11 @@ func newTSODispatcher( tokenCh := make(chan struct{}, tokenChCapacity) td := &tsoDispatcher{ - ctx: dispatcherCtx, - cancel: dispatcherCancel, - provider: provider, - connectionCtxs: &sync.Map{}, - tsoRequestCh: tsoRequestCh, - tsDeadlineCh: make(chan *deadline, tokenChCapacity), + ctx: dispatcherCtx, + cancel: dispatcherCancel, + provider: provider, + tsoRequestCh: tsoRequestCh, + tsDeadlineCh: make(chan *deadline, tokenChCapacity), batchBufferPool: &sync.Pool{ New: func() any { return batch.NewController[*Request]( @@ -137,8 +133,7 @@ func newTSODispatcher( ) }, }, - tokenCh: tokenCh, - updateConnectionCtxsCh: make(chan struct{}, 1), + tokenCh: tokenCh, } go td.watchTSDeadline() return td @@ -168,13 +163,6 @@ func (td *tsoDispatcher) watchTSDeadline() { } } -func (td *tsoDispatcher) scheduleUpdateConnectionCtxs() { - select { - case td.updateConnectionCtxsCh <- struct{}{}: - default: - } -} - func (td *tsoDispatcher) revokePendingRequests(err error) { for range len(td.tsoRequestCh) { req := <-td.tsoRequestCh @@ -196,9 +184,9 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { var ( ctx = td.ctx provider = td.provider - svcDiscovery = provider.getServiceDiscovery() option = provider.getOption() - connectionCtxs = td.connectionCtxs + svcDiscovery = provider.getServiceDiscovery() + conCtxMgr = provider.getConnectionCtxMgr() tsoBatchController *batch.Controller[*Request] ) @@ -207,10 +195,7 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { defer func() { log.Info("[tso] exit tso dispatcher") // Cancel all connections. - connectionCtxs.Range(func(_, cc any) bool { - cc.(*tsoConnectionContext).cancel() - return true - }) + conCtxMgr.ReleaseAll() if tsoBatchController != nil && tsoBatchController.GetCollectedRequestCount() != 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())) @@ -219,8 +204,6 @@ func (td *tsoDispatcher) handleDispatcher(wg *sync.WaitGroup) { td.revokePendingRequests(tsoErr) wg.Done() }() - // Daemon goroutine to update the connectionCtxs periodically and handle the `connectionCtxs` update event. - go td.connectionCtxsUpdater() var ( err error @@ -291,14 +274,14 @@ tsoBatchLoop: // Choose a stream to send the TSO gRPC request. streamChoosingLoop: for { - connectionCtx := chooseStream(connectionCtxs) + connectionCtx := conCtxMgr.GetConnectionCtx() if connectionCtx != nil { - streamCtx, cancel, streamURL, stream = connectionCtx.ctx, connectionCtx.cancel, connectionCtx.streamURL, connectionCtx.stream + streamCtx, cancel, streamURL, stream = connectionCtx.Ctx, connectionCtx.Cancel, connectionCtx.StreamURL, connectionCtx.Stream } // Check stream and retry if necessary. if stream == nil { log.Info("[tso] tso stream is not ready") - if provider.updateConnectionCtxs(ctx, connectionCtxs) { + if provider.updateConnectionCtxs(ctx) { continue streamChoosingLoop } timer := time.NewTimer(constants.RetryInterval) @@ -325,8 +308,7 @@ tsoBatchLoop: case <-streamCtx.Done(): log.Info("[tso] tso stream is canceled", zap.String("stream-url", streamURL)) // Set `stream` to nil and remove this stream from the `connectionCtxs` due to being canceled. - connectionCtxs.Delete(streamURL) - cancel() + conCtxMgr.Release(streamURL) stream = nil continue default: @@ -334,7 +316,7 @@ tsoBatchLoop: // Check if any error has occurred on this stream when receiving asynchronously. if err = stream.GetRecvError(); err != nil { - exit := !td.handleProcessRequestError(ctx, bo, streamURL, cancel, err) + exit := !td.handleProcessRequestError(ctx, bo, conCtxMgr, streamURL, err) stream = nil if exit { td.cancelCollectedRequests(tsoBatchController, invalidStreamID, errors.WithStack(ctx.Err())) @@ -419,7 +401,7 @@ tsoBatchLoop: // reused in the next loop safely. tsoBatchController = nil } else { - exit := !td.handleProcessRequestError(ctx, bo, streamURL, cancel, err) + exit := !td.handleProcessRequestError(ctx, bo, conCtxMgr, streamURL, err) stream = nil if exit { return @@ -430,110 +412,44 @@ tsoBatchLoop: // handleProcessRequestError handles errors occurs when trying to process a TSO RPC request for the dispatcher loop. // Returns true if the dispatcher loop is ok to continue. Otherwise, the dispatcher loop should be exited. -func (td *tsoDispatcher) handleProcessRequestError(ctx context.Context, bo *retry.Backoffer, streamURL string, streamCancelFunc context.CancelFunc, err error) bool { +func (td *tsoDispatcher) handleProcessRequestError( + ctx context.Context, + bo *retry.Backoffer, + conCtxMgr *cctx.Manager[*tsoStream], + streamURL string, + err error, +) bool { + log.Error("[tso] getTS error after processing requests", + zap.String("stream-url", streamURL), + zap.Error(errs.ErrClientGetTSO.FastGenByArgs(err.Error()))) + select { case <-ctx.Done(): return false default: } + // Release this stream from the manager due to error. + conCtxMgr.Release(streamURL) + // Update the member list to ensure the latest topology is used before the next batch. svcDiscovery := td.provider.getServiceDiscovery() - - svcDiscovery.ScheduleCheckMemberChanged() - log.Error("[tso] getTS error after processing requests", - zap.String("stream-url", streamURL), - zap.Error(errs.ErrClientGetTSO.FastGenByArgs(err.Error()))) - // Set `stream` to nil and remove this stream from the `connectionCtxs` due to error. - td.connectionCtxs.Delete(streamURL) - streamCancelFunc() - // Because ScheduleCheckMemberChanged is asynchronous, if the leader changes, we better call `updateMember` ASAP. if errs.IsLeaderChange(err) { + // If the leader changed, we better call `CheckMemberChanged` blockingly to + // ensure the next round of TSO requests can be sent to the new leader. if err := bo.Exec(ctx, svcDiscovery.CheckMemberChanged); err != nil { - select { - case <-ctx.Done(): - return false - default: - } + log.Error("[tso] check member changed error after the leader changed", zap.Error(err)) } - // Because the TSO Follower Proxy could be configured online, - // If we change it from on -> off, background updateConnectionCtxs - // will cancel the current stream, then the EOF error caused by cancel() - // should not trigger the updateConnectionCtxs here. - // So we should only call it when the leader changes. - td.provider.updateConnectionCtxs(ctx, td.connectionCtxs) + } else { + // For other errors, we can just schedule a member change check asynchronously. + svcDiscovery.ScheduleCheckMemberChanged() } - return true -} - -// updateConnectionCtxs updates the `connectionCtxs` regularly. -func (td *tsoDispatcher) connectionCtxsUpdater() { - var ( - ctx = td.ctx - connectionCtxs = td.connectionCtxs - provider = td.provider - option = td.provider.getOption() - updateTicker = &time.Ticker{} - ) - - log.Info("[tso] start tso connection contexts updater") - setNewUpdateTicker := func(interval time.Duration) { - if updateTicker.C != nil { - updateTicker.Stop() - } - if interval == 0 { - updateTicker = &time.Ticker{} - } else { - updateTicker = time.NewTicker(interval) - } - } - // If the TSO Follower Proxy is enabled, set the update interval to the member update interval. - if option.GetEnableTSOFollowerProxy() { - setNewUpdateTicker(sd.MemberUpdateInterval) - } - // Set to nil before returning to ensure that the existing ticker can be GC. - defer setNewUpdateTicker(0) - - for { - provider.updateConnectionCtxs(ctx, connectionCtxs) - select { - case <-ctx.Done(): - log.Info("[tso] exit tso connection contexts updater") - return - case <-option.EnableTSOFollowerProxyCh: - enableTSOFollowerProxy := option.GetEnableTSOFollowerProxy() - log.Info("[tso] tso follower proxy status changed", - zap.Bool("enable", enableTSOFollowerProxy)) - if enableTSOFollowerProxy && updateTicker.C == nil { - // Because the TSO Follower Proxy is enabled, - // the periodic check needs to be performed. - setNewUpdateTicker(sd.MemberUpdateInterval) - } else if !enableTSOFollowerProxy && updateTicker.C != nil { - // Because the TSO Follower Proxy is disabled, - // the periodic check needs to be turned off. - setNewUpdateTicker(0) - } - case <-updateTicker.C: - // Triggered periodically when the TSO Follower Proxy is enabled. - case <-td.updateConnectionCtxsCh: - // Triggered by the leader/follower change. - } - } -} - -// chooseStream uses the reservoir sampling algorithm to randomly choose a connection. -// connectionCtxs will only have only one stream to choose when the TSO Follower Proxy is off. -func chooseStream(connectionCtxs *sync.Map) (connectionCtx *tsoConnectionContext) { - idx := 0 - connectionCtxs.Range(func(_, cc any) bool { - j := rand.Intn(idx + 1) - if j < 1 { - connectionCtx = cc.(*tsoConnectionContext) - } - idx++ + select { + case <-ctx.Done(): + return false + default: return true - }) - return connectionCtx + } } // processRequests sends the RPC request for the batch. It's guaranteed that after calling this function, requests diff --git a/client/clients/tso/dispatcher_test.go b/client/clients/tso/dispatcher_test.go index cefc53f3944..7e5554c7c7b 100644 --- a/client/clients/tso/dispatcher_test.go +++ b/client/clients/tso/dispatcher_test.go @@ -30,19 +30,21 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/client/opt" + cctx "github.com/tikv/pd/client/pkg/connectionctx" sd "github.com/tikv/pd/client/servicediscovery" ) type mockTSOServiceProvider struct { option *opt.Option createStream func(ctx context.Context) *tsoStream - updateConnMu sync.Mutex + conCtxMgr *cctx.Manager[*tsoStream] } func newMockTSOServiceProvider(option *opt.Option, createStream func(ctx context.Context) *tsoStream) *mockTSOServiceProvider { return &mockTSOServiceProvider{ option: option, createStream: createStream, + conCtxMgr: cctx.NewManager[*tsoStream](), } } @@ -54,24 +56,21 @@ func (*mockTSOServiceProvider) getServiceDiscovery() sd.ServiceDiscovery { return sd.NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) } -func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context, 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() +func (m *mockTSOServiceProvider) getConnectionCtxMgr() *cctx.Manager[*tsoStream] { + return m.conCtxMgr +} - _, ok := connectionCtxs.Load(mockStreamURL) - if ok { +func (m *mockTSOServiceProvider) updateConnectionCtxs(ctx context.Context) bool { + if m.conCtxMgr.Exist(mockStreamURL) { return true } - ctx, cancel := context.WithCancel(ctx) 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}) + m.conCtxMgr.Store(ctx, mockStreamURL, stream) return true } diff --git a/client/pkg/connectionctx/manager.go b/client/pkg/connectionctx/manager.go new file mode 100644 index 00000000000..04c1eb13d3a --- /dev/null +++ b/client/pkg/connectionctx/manager.go @@ -0,0 +1,143 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package connectionctx + +import ( + "context" + "sync" + + "golang.org/x/exp/rand" +) + +type connectionCtx[T any] struct { + Ctx context.Context + Cancel context.CancelFunc + // Current URL of the stream connection. + StreamURL string + // Current stream to send the gRPC requests. + Stream T +} + +// Manager is used to manage the connection contexts. +type Manager[T any] struct { + sync.RWMutex + connectionCtxs map[string]*connectionCtx[T] +} + +// NewManager is used to create a new connection context manager. +func NewManager[T any]() *Manager[T] { + return &Manager[T]{ + connectionCtxs: make(map[string]*connectionCtx[T], 3), + } +} + +// Exist is used to check if the connection context exists by the given URL. +func (c *Manager[T]) Exist(url string) bool { + c.RLock() + defer c.RUnlock() + _, ok := c.connectionCtxs[url] + return ok +} + +// Store is used to store the connection context, `overwrite` is used to force the store operation +// no matter whether the connection context exists before, which is false by default. +func (c *Manager[T]) Store(ctx context.Context, url string, stream T, overwrite ...bool) { + c.Lock() + defer c.Unlock() + overwriteFlag := false + if len(overwrite) > 0 { + overwriteFlag = overwrite[0] + } + _, ok := c.connectionCtxs[url] + if !overwriteFlag && ok { + return + } + c.storeLocked(ctx, url, stream) +} + +func (c *Manager[T]) storeLocked(ctx context.Context, url string, stream T) { + c.releaseLocked(url) + cctx, cancel := context.WithCancel(ctx) + c.connectionCtxs[url] = &connectionCtx[T]{cctx, cancel, url, stream} +} + +// CleanAllAndStore is used to store the connection context exclusively. It will release +// all other connection contexts. `stream` is optional, if it is not provided, all +// connection contexts other than the given `url` will be released. +func (c *Manager[T]) CleanAllAndStore(ctx context.Context, url string, stream ...T) { + c.Lock() + defer c.Unlock() + // Remove all other `connectionCtx`s. + c.gcLocked(func(curURL string) bool { + return curURL != url + }) + if len(stream) == 0 { + return + } + c.storeLocked(ctx, url, stream[0]) +} + +// GC is used to release all connection contexts that match the given condition. +func (c *Manager[T]) GC(condition func(url string) bool) { + c.Lock() + defer c.Unlock() + c.gcLocked(condition) +} + +func (c *Manager[T]) gcLocked(condition func(url string) bool) { + for url := range c.connectionCtxs { + if condition(url) { + c.releaseLocked(url) + } + } +} + +// ReleaseAll is used to release all connection contexts. +func (c *Manager[T]) ReleaseAll() { + c.GC(func(string) bool { return true }) +} + +// Release is used to delete a connection context from the connection context map and release the resources. +func (c *Manager[T]) Release(url string) { + c.Lock() + defer c.Unlock() + c.releaseLocked(url) +} + +func (c *Manager[T]) releaseLocked(url string) { + cc, ok := c.connectionCtxs[url] + if !ok { + return + } + cc.Cancel() + delete(c.connectionCtxs, url) +} + +// GetConnectionCtx is used to get a connection context from the connection context map. +// It uses the reservoir sampling algorithm to randomly pick one connection context. +func (c *Manager[T]) GetConnectionCtx() *connectionCtx[T] { + c.RLock() + defer c.RUnlock() + idx := 0 + var connectionCtx *connectionCtx[T] + for _, cc := range c.connectionCtxs { + j := rand.Intn(idx + 1) + if j < 1 { + connectionCtx = cc + } + idx++ + } + return connectionCtx +} diff --git a/client/pkg/connectionctx/manager_test.go b/client/pkg/connectionctx/manager_test.go new file mode 100644 index 00000000000..42504673b95 --- /dev/null +++ b/client/pkg/connectionctx/manager_test.go @@ -0,0 +1,83 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package connectionctx + +import ( + "context" + "fmt" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestManager(t *testing.T) { + re := require.New(t) + ctx := context.Background() + manager := NewManager[int]() + + re.False(manager.Exist("test-url")) + manager.Store(ctx, "test-url", 1) + re.True(manager.Exist("test-url")) + + cctx := manager.GetConnectionCtx() + re.Equal("test-url", cctx.StreamURL) + re.Equal(1, cctx.Stream) + + manager.Store(ctx, "test-url", 2) + cctx = manager.GetConnectionCtx() + re.Equal("test-url", cctx.StreamURL) + re.Equal(1, cctx.Stream) + + manager.Store(ctx, "test-url", 2, true) + cctx = manager.GetConnectionCtx() + re.Equal("test-url", cctx.StreamURL) + re.Equal(2, cctx.Stream) + + manager.Store(ctx, "test-another-url", 3) + pickedCount := make(map[string]int) + for range 1000 { + cctx = manager.GetConnectionCtx() + pickedCount[cctx.StreamURL]++ + } + re.NotEmpty(pickedCount["test-url"]) + re.NotEmpty(pickedCount["test-another-url"]) + re.Equal(1000, pickedCount["test-url"]+pickedCount["test-another-url"]) + + manager.GC(func(url string) bool { + return url == "test-url" + }) + re.False(manager.Exist("test-url")) + re.True(manager.Exist("test-another-url")) + + manager.CleanAllAndStore(ctx, "test-url", 1) + re.True(manager.Exist("test-url")) + re.False(manager.Exist("test-another-url")) + + manager.Store(ctx, "test-another-url", 3) + manager.CleanAllAndStore(ctx, "test-unique-url", 4) + re.True(manager.Exist("test-unique-url")) + re.False(manager.Exist("test-url")) + re.False(manager.Exist("test-another-url")) + + manager.Release("test-unique-url") + re.False(manager.Exist("test-unique-url")) + + for i := range 1000 { + manager.Store(ctx, fmt.Sprintf("test-url-%d", i), i) + } + re.Len(manager.connectionCtxs, 1000) + manager.ReleaseAll() + re.Empty(manager.connectionCtxs) +} diff --git a/client/servicediscovery/pd_service_discovery.go b/client/servicediscovery/pd_service_discovery.go index 619d4196408..5530f3cfa9b 100644 --- a/client/servicediscovery/pd_service_discovery.go +++ b/client/servicediscovery/pd_service_discovery.go @@ -966,12 +966,9 @@ func (c *pdServiceDiscovery) updateURLs(members []*pdpb.Member) { return } c.urls.Store(urls) - // Update the connection contexts when member changes if TSO Follower Proxy is enabled. - if c.option.GetEnableTSOFollowerProxy() { - // Run callbacks to reflect the membership changes in the leader and followers. - for _, cb := range c.membersChangedCbs { - cb() - } + // Run callbacks to reflect the membership changes in the leader and followers. + for _, cb := range c.membersChangedCbs { + cb() } log.Info("[pd] update member urls", zap.Strings("old-urls", oldURLs), zap.Strings("new-urls", urls)) } From 5999d6460d4261b0a1f63c3b5f379167b694229a Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Fri, 27 Dec 2024 17:28:06 +0800 Subject: [PATCH 23/33] *: fix lint issue (#8953) ref tikv/pd#4322 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/cgroup/cgroup_cpu_test.go | 5 ++--- pkg/core/region_test.go | 4 ++-- pkg/mcs/resourcemanager/server/metrics_test.go | 5 ++--- pkg/mcs/resourcemanager/server/token_buckets_test.go | 7 +++---- pkg/tso/keyspace_group_manager_test.go | 6 +++--- pkg/window/policy_test.go | 9 ++++----- tests/integrations/realcluster/cluster.go | 3 +-- tests/server/api/api_test.go | 4 ++-- tools/pd-ctl/tests/scheduler/scheduler_test.go | 12 ++++++------ 9 files changed, 25 insertions(+), 30 deletions(-) diff --git a/pkg/cgroup/cgroup_cpu_test.go b/pkg/cgroup/cgroup_cpu_test.go index 441c2192e79..6d4d8f39f49 100644 --- a/pkg/cgroup/cgroup_cpu_test.go +++ b/pkg/cgroup/cgroup_cpu_test.go @@ -17,7 +17,6 @@ package cgroup import ( - "fmt" "regexp" "runtime" "strconv" @@ -45,10 +44,10 @@ func checkKernelVersionNewerThan(re *require.Assertions, t *testing.T, major, mi t.Log("kernel release string:", releaseStr) versionInfoRE := regexp.MustCompile(`[0-9]+\.[0-9]+\.[0-9]+`) kernelVersion := versionInfoRE.FindAllString(releaseStr, 1) - re.Len(kernelVersion, 1, fmt.Sprintf("release str is %s", releaseStr)) + re.Lenf(kernelVersion, 1, "release str is %s", releaseStr) kernelVersionPartRE := regexp.MustCompile(`[0-9]+`) kernelVersionParts := kernelVersionPartRE.FindAllString(kernelVersion[0], -1) - re.Len(kernelVersionParts, 3, fmt.Sprintf("kernel version str is %s", kernelVersion[0])) + re.Lenf(kernelVersionParts, 3, "kernel version str is %s", kernelVersion[0]) t.Logf("parsed kernel version parts: major %s, minor %s, patch %s", kernelVersionParts[0], kernelVersionParts[1], kernelVersionParts[2]) mustConvInt := func(s string) int { diff --git a/pkg/core/region_test.go b/pkg/core/region_test.go index 51ba5fe96dc..473421b0e52 100644 --- a/pkg/core/region_test.go +++ b/pkg/core/region_test.go @@ -985,10 +985,10 @@ func TestUpdateRegionEquivalence(t *testing.T) { checkRegions(re, regionsNew) for _, r := range regionsOld.GetRegions() { - re.Equal(int32(2), r.GetRef(), fmt.Sprintf("inconsistent region %d", r.GetID())) + re.Equalf(int32(2), r.GetRef(), "inconsistent region %d", r.GetID()) } for _, r := range regionsNew.GetRegions() { - re.Equal(int32(2), r.GetRef(), fmt.Sprintf("inconsistent region %d", r.GetID())) + re.Equalf(int32(2), r.GetRef(), "inconsistent region %d", r.GetID()) } for i := 1; i <= storeNums; i++ { diff --git a/pkg/mcs/resourcemanager/server/metrics_test.go b/pkg/mcs/resourcemanager/server/metrics_test.go index 4c3ec7ce5ef..97b21bf5ce3 100644 --- a/pkg/mcs/resourcemanager/server/metrics_test.go +++ b/pkg/mcs/resourcemanager/server/metrics_test.go @@ -15,7 +15,6 @@ package server import ( - "fmt" "testing" "github.com/stretchr/testify/require" @@ -43,8 +42,8 @@ func TestMaxPerSecCostTracker(t *testing.T) { // Check the max values at the end of each flushPeriod if (i+1)%20 == 0 { period := i / 20 - re.Equal(tracker.maxPerSecRRU, expectedMaxRU[period], fmt.Sprintf("maxPerSecRRU in period %d is incorrect", period+1)) - re.Equal(tracker.maxPerSecWRU, expectedMaxRU[period], fmt.Sprintf("maxPerSecWRU in period %d is incorrect", period+1)) + re.Equalf(tracker.maxPerSecRRU, expectedMaxRU[period], "maxPerSecRRU in period %d is incorrect", period+1) + re.Equalf(tracker.maxPerSecWRU, expectedMaxRU[period], "maxPerSecWRU in period %d is incorrect", period+1) re.Equal(tracker.rruSum, expectedSum[period]) re.Equal(tracker.rruSum, expectedSum[period]) } diff --git a/pkg/mcs/resourcemanager/server/token_buckets_test.go b/pkg/mcs/resourcemanager/server/token_buckets_test.go index b56ccb6ab96..676b1127f35 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets_test.go +++ b/pkg/mcs/resourcemanager/server/token_buckets_test.go @@ -15,7 +15,6 @@ package server import ( - "fmt" "math" "testing" "time" @@ -182,9 +181,9 @@ func TestGroupTokenBucketRequestLoop(t *testing.T) { currentTime := initialTime for i, tc := range testCases { tb, trickle := gtb.request(currentTime, tc.requestTokens, uint64(targetPeriod)/uint64(time.Millisecond), clientUniqueID) - re.Equal(tc.globalBucketTokensAfterAssign, gtb.GetTokenBucket().Tokens, fmt.Sprintf("Test case %d failed: expected bucket tokens %f, got %f", i, tc.globalBucketTokensAfterAssign, gtb.GetTokenBucket().Tokens)) - re.LessOrEqual(math.Abs(tb.Tokens-tc.assignedTokens), 1e-7, fmt.Sprintf("Test case %d failed: expected tokens %f, got %f", i, tc.assignedTokens, tb.Tokens)) - re.Equal(tc.expectedTrickleMs, trickle, fmt.Sprintf("Test case %d failed: expected trickle %d, got %d", i, tc.expectedTrickleMs, trickle)) + re.Equalf(tc.globalBucketTokensAfterAssign, gtb.GetTokenBucket().Tokens, "Test case %d failed: expected bucket tokens %f, got %f", i, tc.globalBucketTokensAfterAssign, gtb.GetTokenBucket().Tokens) + re.LessOrEqualf(math.Abs(tb.Tokens-tc.assignedTokens), 1e-7, "Test case %d failed: expected tokens %f, got %f", i, tc.assignedTokens, tb.Tokens) + re.Equalf(tc.expectedTrickleMs, trickle, "Test case %d failed: expected trickle %d, got %d", i, tc.expectedTrickleMs, trickle) currentTime = currentTime.Add(timeIncrement) } } diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index b4393a23471..c54bbcc1b33 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -886,7 +886,7 @@ func collectAssignedKeyspaceGroupIDs(re *require.Assertions, kgm *KeyspaceGroupM for i := range kgm.kgs { kg := kgm.kgs[i] if kg == nil { - re.Nil(kgm.ams[i], fmt.Sprintf("ksg is nil but am is not nil for id %d", i)) + re.Nilf(kgm.ams[i], "ksg is nil but am is not nil for id %d", i) } else { am := kgm.ams[i] if am != nil { @@ -976,8 +976,8 @@ func (suite *keyspaceGroupManagerTestSuite) TestUpdateKeyspaceGroupMembership() func verifyLocalKeyspaceLookupTable( re *require.Assertions, keyspaceLookupTable map[uint32]struct{}, newKeyspaces []uint32, ) { - re.Equal(len(newKeyspaces), len(keyspaceLookupTable), - fmt.Sprintf("%v %v", newKeyspaces, keyspaceLookupTable)) + re.Equalf(len(newKeyspaces), len(keyspaceLookupTable), + "%v %v", newKeyspaces, keyspaceLookupTable) for _, keyspace := range newKeyspaces { _, ok := keyspaceLookupTable[keyspace] re.True(ok) diff --git a/pkg/window/policy_test.go b/pkg/window/policy_test.go index 936360ccb2b..b5a04c03e4b 100644 --- a/pkg/window/policy_test.go +++ b/pkg/window/policy_test.go @@ -18,7 +18,6 @@ package window import ( - "fmt" "math" "testing" "time" @@ -79,11 +78,11 @@ func TestRollingPolicy_Add(t *testing.T) { asExpected = false } if asExpected { - re.Less(math.Abs(point-policy.window.buckets[offset].Points[0]), 1e-6, - fmt.Sprintf("error, time since last append: %vms, last offset: %v", totalTS, lastOffset)) + re.Lessf(math.Abs(point-policy.window.buckets[offset].Points[0]), 1e-6, + "error, time since last append: %vms, last offset: %v", totalTS, lastOffset) } - re.Less(math.Abs(points[i]-policy.window.buckets[offset].Points[0]), 1e-6, - fmt.Sprintf("error, time since last append: %vms, last offset: %v", totalTS, lastOffset)) + re.Lessf(math.Abs(points[i]-policy.window.buckets[offset].Points[0]), 1e-6, + "error, time since last append: %vms, last offset: %v", totalTS, lastOffset) lastOffset = offset } }) diff --git a/tests/integrations/realcluster/cluster.go b/tests/integrations/realcluster/cluster.go index cc4f6b54713..fc5d1bc4441 100644 --- a/tests/integrations/realcluster/cluster.go +++ b/tests/integrations/realcluster/cluster.go @@ -185,6 +185,5 @@ func waitTiupReady(t *testing.T, tag string) { zap.String("tag", tag), zap.Error(err)) time.Sleep(time.Duration(interval) * time.Second) } - // this check can trigger the cleanup function - require.NotZero(t, 1, "TiUP is not ready", "tag: %s", tag) + require.FailNowf(t, "TiUP is not ready after retry: %s", tag) } diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index 00c43d11309..faa22ce08f4 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -173,8 +173,8 @@ func (suite *middlewareTestSuite) TestRequestInfoMiddleware() { re.Equal(http.StatusOK, resp.StatusCode) re.Equal("Profile", resp.Header.Get("service-label")) - re.Equal("{\"seconds\":[\"1\"]}", resp.Header.Get("url-param")) - re.Equal("{\"testkey\":\"testvalue\"}", resp.Header.Get("body-param")) + re.JSONEq("{\"seconds\":[\"1\"]}", resp.Header.Get("url-param")) + re.JSONEq("{\"testkey\":\"testvalue\"}", resp.Header.Get("body-param")) re.Equal("HTTP/1.1/POST:/pd/api/v1/debug/pprof/profile", resp.Header.Get("method")) re.Equal("anonymous", resp.Header.Get("caller-id")) re.Equal("127.0.0.1", resp.Header.Get("ip")) diff --git a/tools/pd-ctl/tests/scheduler/scheduler_test.go b/tools/pd-ctl/tests/scheduler/scheduler_test.go index 787bdaa4521..f3a81845921 100644 --- a/tools/pd-ctl/tests/scheduler/scheduler_test.go +++ b/tools/pd-ctl/tests/scheduler/scheduler_test.go @@ -186,18 +186,18 @@ func (suite *schedulerTestSuite) checkScheduler(cluster *pdTests.TestCluster) { case "grant-leader-scheduler": return "paused", !storeInfo.AllowLeaderTransferOut() default: - re.Fail(fmt.Sprintf("unknown scheduler %s", schedulerName)) + re.Failf("unknown scheduler %s", schedulerName) return "", false } }() if slice.AnyOf(changedStores, func(i int) bool { return store.GetId() == changedStores[i] }) { - re.True(isStorePaused, - fmt.Sprintf("store %d should be %s with %s", store.GetId(), status, schedulerName)) + re.Truef(isStorePaused, + "store %d should be %s with %s", store.GetId(), status, schedulerName) } else { - re.False(isStorePaused, - fmt.Sprintf("store %d should not be %s with %s", store.GetId(), status, schedulerName)) + re.Falsef(isStorePaused, + "store %d should not be %s with %s", store.GetId(), status, schedulerName) } if sche := cluster.GetSchedulingPrimaryServer(); sche != nil { switch schedulerName { @@ -206,7 +206,7 @@ func (suite *schedulerTestSuite) checkScheduler(cluster *pdTests.TestCluster) { case "grant-leader-scheduler": re.Equal(isStorePaused, !sche.GetCluster().GetStore(store.GetId()).AllowLeaderTransferOut()) default: - re.Fail(fmt.Sprintf("unknown scheduler %s", schedulerName)) + re.Failf("unknown scheduler %s", schedulerName) } } } From 7d33065019f78d9150a8c89ddb4593f81e6ff9b3 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Tue, 31 Dec 2024 11:15:42 +0800 Subject: [PATCH 24/33] grafana: Add `gRPC Received commands rate` panel (#8921) close tikv/pd#8920 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- metrics/grafana/pd.json | 220 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 220 insertions(+) diff --git a/metrics/grafana/pd.json b/metrics/grafana/pd.json index 0f4e91afd50..62b2e7234ef 100644 --- a/metrics/grafana/pd.json +++ b/metrics/grafana/pd.json @@ -8937,6 +8937,226 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "dashLength": 10, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The rate of received each kind of gRPC commands", + "editable": true, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 127 + }, + "id": 904, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "paceLength": 10, + "pointradius": 5, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "targets": [ + { + "expr": "sum(rate(grpc_server_msg_received_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, grpc_method)", + "legendFormat": "{{instance}}-{{grpc_method}}", + "interval": "", + "exemplar": true, + "intervalFactor": 2, + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeRegions": [], + "title": "gRPC Received commands rate", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "label": null, + "logBase": 10, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + }, + "options": { + "alertThreshold": true + }, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "pluginVersion": "7.5.17", + "bars": false, + "dashes": false, + "decimals": null, + "error": false, + "percentage": false, + "points": false, + "stack": false, + "steppedLine": false, + "timeFrom": null, + "timeShift": null, + "fillGradient": 0, + "hiddenSeries": false + }, + { + "aliasColors": {}, + "dashLength": 10, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The error rate of handled gRPC commands.Note: It can't catch the error hide in the header, like this https://github.com/tikv/pd/blob/2d970a619a8917c35d306f401326141481c133e0/server/grpc_service.go#L2071", + "editable": true, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 135 + }, + "id": 905, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "paceLength": 10, + "pluginVersion": "7.5.17", + "pointradius": 5, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "targets": [ + { + "expr": "sum(rate(grpc_server_handled_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", grpc_type=\"unary\", grpc_code!=\"OK\"}[1m])) by (grpc_method)", + "legendFormat": "{{grpc_method}}", + "interval": "", + "exemplar": true, + "intervalFactor": 2, + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeRegions": [], + "title": "gRPC Error rate", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "label": null, + "logBase": 10, + "max": null, + "min": null, + "show": true, + "$$hashKey": "object:132" + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true, + "$$hashKey": "object:133" + } + ], + "yaxis": { + "align": false, + "alignLevel": null + }, + "bars": false, + "dashes": false, + "decimals": null, + "error": false, + "fillGradient": 0, + "hiddenSeries": false, + "percentage": false, + "points": false, + "stack": false, + "steppedLine": false, + "timeFrom": null, + "timeShift": null } ], "repeat": null, From 5ad4301b7a095a348ba9d5a5ffd2df8d021dcf44 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 31 Dec 2024 13:54:33 +0800 Subject: [PATCH 25/33] client/pkg: introduce the deadline watcher (#8955) ref tikv/pd#8690 Introduce the deadline watcher. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/clients/tso/dispatcher.go | 73 +++--------------- client/errs/errno.go | 1 - client/pkg/deadline/watcher.go | 111 ++++++++++++++++++++++++++++ client/pkg/deadline/watcher_test.go | 58 +++++++++++++++ errors.toml | 5 -- pkg/errs/errno.go | 1 - 6 files changed, 181 insertions(+), 68 deletions(-) create mode 100644 client/pkg/deadline/watcher.go create mode 100644 client/pkg/deadline/watcher_test.go diff --git a/client/clients/tso/dispatcher.go b/client/clients/tso/dispatcher.go index c05ab27d755..1cc2b2aa940 100644 --- a/client/clients/tso/dispatcher.go +++ b/client/clients/tso/dispatcher.go @@ -36,33 +36,12 @@ import ( "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/batch" cctx "github.com/tikv/pd/client/pkg/connectionctx" + "github.com/tikv/pd/client/pkg/deadline" "github.com/tikv/pd/client/pkg/retry" - "github.com/tikv/pd/client/pkg/utils/timerutil" "github.com/tikv/pd/client/pkg/utils/tsoutil" sd "github.com/tikv/pd/client/servicediscovery" ) -// deadline is used to control the TS request timeout manually, -// it will be sent to the `tsDeadlineCh` to be handled by the `watchTSDeadline` goroutine. -type deadline struct { - timer *time.Timer - done chan struct{} - cancel context.CancelFunc -} - -func newTSDeadline( - timeout time.Duration, - done chan struct{}, - cancel context.CancelFunc, -) *deadline { - timer := timerutil.GlobalTimerPool.Get(timeout) - return &deadline{ - timer: timer, - done: done, - cancel: cancel, - } -} - type tsoInfo struct { tsoServer string reqKeyspaceGroupID uint32 @@ -86,10 +65,10 @@ type tsoDispatcher struct { ctx context.Context cancel context.CancelFunc - provider tsoServiceProvider - tsoRequestCh chan *Request - tsDeadlineCh chan *deadline - latestTSOInfo atomic.Pointer[tsoInfo] + provider tsoServiceProvider + tsoRequestCh chan *Request + deadlineWatcher *deadline.Watcher + latestTSOInfo atomic.Pointer[tsoInfo] // For reusing `*batchController` objects batchBufferPool *sync.Pool @@ -119,11 +98,11 @@ func newTSODispatcher( tokenCh := make(chan struct{}, tokenChCapacity) td := &tsoDispatcher{ - ctx: dispatcherCtx, - cancel: dispatcherCancel, - provider: provider, - tsoRequestCh: tsoRequestCh, - tsDeadlineCh: make(chan *deadline, tokenChCapacity), + ctx: dispatcherCtx, + cancel: dispatcherCancel, + provider: provider, + tsoRequestCh: tsoRequestCh, + deadlineWatcher: deadline.NewWatcher(dispatcherCtx, tokenChCapacity, "tso"), batchBufferPool: &sync.Pool{ New: func() any { return batch.NewController[*Request]( @@ -135,34 +114,9 @@ func newTSODispatcher( }, tokenCh: tokenCh, } - go td.watchTSDeadline() return td } -func (td *tsoDispatcher) watchTSDeadline() { - log.Info("[tso] start tso deadline watcher") - defer log.Info("[tso] exit tso deadline watcher") - for { - select { - case d := <-td.tsDeadlineCh: - select { - case <-d.timer.C: - log.Error("[tso] tso request is canceled due to timeout", - errs.ZapError(errs.ErrClientGetTSOTimeout)) - d.cancel() - timerutil.GlobalTimerPool.Put(d.timer) - case <-d.done: - timerutil.GlobalTimerPool.Put(d.timer) - case <-td.ctx.Done(): - timerutil.GlobalTimerPool.Put(d.timer) - return - } - case <-td.ctx.Done(): - return - } - } -} - func (td *tsoDispatcher) revokePendingRequests(err error) { for range len(td.tsoRequestCh) { req := <-td.tsoRequestCh @@ -378,14 +332,11 @@ tsoBatchLoop: } } - done := make(chan struct{}) - dl := newTSDeadline(option.Timeout, done, cancel) - select { - case <-ctx.Done(): + done := td.deadlineWatcher.Start(ctx, option.Timeout, cancel) + if done == nil { // Finish the collected requests if the context is canceled. td.cancelCollectedRequests(tsoBatchController, invalidStreamID, errors.WithStack(ctx.Err())) return - case td.tsDeadlineCh <- dl: } // processRequests guarantees that the collected requests could be finished properly. err = td.processRequests(stream, tsoBatchController, done) diff --git a/client/errs/errno.go b/client/errs/errno.go index 25665f01017..99a426d0776 100644 --- a/client/errs/errno.go +++ b/client/errs/errno.go @@ -56,7 +56,6 @@ var ( ErrClientGetMetaStorageClient = errors.Normalize("failed to get meta storage client", errors.RFCCodeText("PD:client:ErrClientGetMetaStorageClient")) ErrClientCreateTSOStream = errors.Normalize("create TSO stream failed, %s", errors.RFCCodeText("PD:client:ErrClientCreateTSOStream")) ErrClientTSOStreamClosed = errors.Normalize("encountered TSO stream being closed unexpectedly", errors.RFCCodeText("PD:client:ErrClientTSOStreamClosed")) - ErrClientGetTSOTimeout = errors.Normalize("get TSO timeout", errors.RFCCodeText("PD:client:ErrClientGetTSOTimeout")) ErrClientGetTSO = errors.Normalize("get TSO failed, %v", errors.RFCCodeText("PD:client:ErrClientGetTSO")) ErrClientGetMinTSO = errors.Normalize("get min TSO failed, %v", errors.RFCCodeText("PD:client:ErrClientGetMinTSO")) ErrClientGetLeader = errors.Normalize("get leader failed, %v", errors.RFCCodeText("PD:client:ErrClientGetLeader")) diff --git a/client/pkg/deadline/watcher.go b/client/pkg/deadline/watcher.go new file mode 100644 index 00000000000..b40857edbfd --- /dev/null +++ b/client/pkg/deadline/watcher.go @@ -0,0 +1,111 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package deadline + +import ( + "context" + "time" + + "go.uber.org/zap" + + "github.com/pingcap/log" + + "github.com/tikv/pd/client/pkg/utils/timerutil" +) + +// The `cancel` function will be invoked once the specified `timeout` elapses without receiving a `done` signal. +type deadline struct { + timer *time.Timer + done chan struct{} + cancel context.CancelFunc +} + +// Watcher is used to watch and manage the deadlines. +type Watcher struct { + ctx context.Context + source string + Ch chan *deadline +} + +// NewWatcher is used to create a new deadline watcher. +func NewWatcher(ctx context.Context, capacity int, source string) *Watcher { + watcher := &Watcher{ + ctx: ctx, + source: source, + Ch: make(chan *deadline, capacity), + } + go watcher.Watch() + return watcher +} + +// Watch is used to watch the deadlines and invoke the `cancel` function when the deadline is reached. +// The `err` will be returned if the deadline is reached. +func (w *Watcher) Watch() { + log.Info("[pd] start the deadline watcher", zap.String("source", w.source)) + defer log.Info("[pd] exit the deadline watcher", zap.String("source", w.source)) + for { + select { + case d := <-w.Ch: + select { + case <-d.timer.C: + log.Error("[pd] the deadline is reached", zap.String("source", w.source)) + d.cancel() + timerutil.GlobalTimerPool.Put(d.timer) + case <-d.done: + timerutil.GlobalTimerPool.Put(d.timer) + case <-w.ctx.Done(): + timerutil.GlobalTimerPool.Put(d.timer) + return + } + case <-w.ctx.Done(): + return + } + } +} + +// Start is used to start a deadline. It returns a channel which will be closed when the deadline is reached. +// Returns nil if the deadline is not started. +func (w *Watcher) Start( + ctx context.Context, + timeout time.Duration, + cancel context.CancelFunc, +) chan struct{} { + // Check if the watcher is already canceled. + select { + case <-w.ctx.Done(): + return nil + case <-ctx.Done(): + return nil + default: + } + // Initialize the deadline. + timer := timerutil.GlobalTimerPool.Get(timeout) + d := &deadline{ + timer: timer, + done: make(chan struct{}), + cancel: cancel, + } + // Send the deadline to the watcher. + select { + case <-w.ctx.Done(): + timerutil.GlobalTimerPool.Put(timer) + return nil + case <-ctx.Done(): + timerutil.GlobalTimerPool.Put(timer) + return nil + case w.Ch <- d: + return d.done + } +} diff --git a/client/pkg/deadline/watcher_test.go b/client/pkg/deadline/watcher_test.go new file mode 100644 index 00000000000..b93987b8874 --- /dev/null +++ b/client/pkg/deadline/watcher_test.go @@ -0,0 +1,58 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package deadline + +import ( + "context" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestWatcher(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + watcher := NewWatcher(ctx, 10, "test") + var deadlineReached atomic.Bool + done := watcher.Start(ctx, time.Millisecond, func() { + deadlineReached.Store(true) + }) + re.NotNil(done) + time.Sleep(5 * time.Millisecond) + re.True(deadlineReached.Load()) + + deadlineReached.Store(false) + done = watcher.Start(ctx, 500*time.Millisecond, func() { + deadlineReached.Store(true) + }) + re.NotNil(done) + done <- struct{}{} + time.Sleep(time.Second) + re.False(deadlineReached.Load()) + + deadCtx, deadCancel := context.WithCancel(ctx) + deadCancel() + deadlineReached.Store(false) + done = watcher.Start(deadCtx, time.Millisecond, func() { + deadlineReached.Store(true) + }) + re.Nil(done) + time.Sleep(5 * time.Millisecond) + re.False(deadlineReached.Load()) +} diff --git a/errors.toml b/errors.toml index 2ab3b014f5a..9980a98ab14 100644 --- a/errors.toml +++ b/errors.toml @@ -131,11 +131,6 @@ error = ''' get TSO failed ''' -["PD:client:ErrClientGetTSOTimeout"] -error = ''' -get TSO timeout -''' - ["PD:cluster:ErrInvalidStoreID"] error = ''' invalid store id %d, not found diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index ee24b4d0673..834bf4f824e 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -144,7 +144,6 @@ var ( // client errors var ( ErrClientCreateTSOStream = errors.Normalize("create TSO stream failed, %s", errors.RFCCodeText("PD:client:ErrClientCreateTSOStream")) - ErrClientGetTSOTimeout = errors.Normalize("get TSO timeout", errors.RFCCodeText("PD:client:ErrClientGetTSOTimeout")) ErrClientGetTSO = errors.Normalize("get TSO failed", errors.RFCCodeText("PD:client:ErrClientGetTSO")) ErrClientGetLeader = errors.Normalize("get leader failed, %v", errors.RFCCodeText("PD:client:ErrClientGetLeader")) ErrClientGetMember = errors.Normalize("get member failed", errors.RFCCodeText("PD:client:ErrClientGetMember")) From c2d48542d8c4c1d1a42ebc0b2449993080d57d29 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 2 Jan 2025 11:20:24 +0800 Subject: [PATCH 26/33] *: format log fields using kebab-case style (#8956) ref tikv/pd#4322 Signed-off-by: Ryan Leung --- pkg/core/region.go | 4 ++-- pkg/encryption/key_manager.go | 2 +- pkg/keyspace/keyspace.go | 2 +- .../resourcemanager/server/token_buckets.go | 2 +- pkg/memory/meminfo.go | 4 ++-- pkg/schedule/config/store_config.go | 2 +- pkg/schedule/schedulers/grant_hot_region.go | 2 +- pkg/tso/global_allocator.go | 2 +- pkg/tso/keyspace_group_manager.go | 22 +++++++++---------- server/grpc_service.go | 2 +- server/server.go | 2 +- server/util.go | 2 +- 12 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pkg/core/region.go b/pkg/core/region.go index 5f5a4a5f2e0..706e6bbd712 100644 --- a/pkg/core/region.go +++ b/pkg/core/region.go @@ -1885,7 +1885,7 @@ func scanRegion(regionTree *regionTree, keyRange *KeyRange, limit int, outputMus keyRange.StartKey, keyRange.EndKey, lastRegion.GetStartKey(), lastRegion.GetEndKey(), region.GetStartKey(), region.GetEndKey()) - log.Warn("scan regions failed", zap.Bool("outputMustContainAllKeyRange", + log.Warn("scan regions failed", zap.Bool("contain-all-key-range", outputMustContainAllKeyRange), zap.Error(err)) if outputMustContainAllKeyRange { return false @@ -1907,7 +1907,7 @@ func scanRegion(regionTree *regionTree, keyRange *KeyRange, limit int, outputMus keyRange.StartKey, keyRange.EndKey, lastRegion.GetStartKey(), lastRegion.GetEndKey(), lastRegion.GetEndKey(), keyRange.EndKey) - log.Warn("scan regions failed", zap.Bool("outputMustContainAllKeyRange", + log.Warn("scan regions failed", zap.Bool("contain-all-key-range", outputMustContainAllKeyRange), zap.Error(err)) if outputMustContainAllKeyRange { return nil, err diff --git a/pkg/encryption/key_manager.go b/pkg/encryption/key_manager.go index 54e5fa01b35..5fc6788c549 100644 --- a/pkg/encryption/key_manager.go +++ b/pkg/encryption/key_manager.go @@ -413,7 +413,7 @@ func (m *Manager) rotateKeyIfNeeded(forceUpdate bool) error { keys.Keys[keyID] = key keys.CurrentKeyId = keyID rotated = true - log.Info("ready to create or rotate data encryption key", zap.Uint64("keyID", keyID)) + log.Info("ready to create or rotate data encryption key", zap.Uint64("key-id", keyID)) break } // Duplicated key id. retry. diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index c9e390df47a..93312ae4ff1 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -593,7 +593,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return nil, err } log.Info("[keyspace] keyspace state updated", - zap.Uint32("ID", meta.GetId()), + zap.Uint32("id", meta.GetId()), zap.String("keyspace-id", meta.GetName()), zap.String("new-state", newState.String()), ) diff --git a/pkg/mcs/resourcemanager/server/token_buckets.go b/pkg/mcs/resourcemanager/server/token_buckets.go index 50dc78c9d68..be32884e1ea 100644 --- a/pkg/mcs/resourcemanager/server/token_buckets.go +++ b/pkg/mcs/resourcemanager/server/token_buckets.go @@ -174,7 +174,7 @@ func (gts *GroupTokenBucketState) balanceSlotTokens( if time.Since(slot.lastReqTime) >= slotExpireTimeout { delete(gts.tokenSlots, clientUniqueID) log.Info("delete resource group slot because expire", zap.Time("last-req-time", slot.lastReqTime), - zap.Any("expire timeout", slotExpireTimeout), zap.Any("del client id", clientUniqueID), zap.Any("len", len(gts.tokenSlots))) + zap.Duration("expire-timeout", slotExpireTimeout), zap.Uint64("del-client-id", clientUniqueID), zap.Int("len", len(gts.tokenSlots))) } } } diff --git a/pkg/memory/meminfo.go b/pkg/memory/meminfo.go index 7ed1afb579b..9ce9ff9b886 100644 --- a/pkg/memory/meminfo.go +++ b/pkg/memory/meminfo.go @@ -210,9 +210,9 @@ func InitMemoryHook() { MemTotal = MemTotalCGroup MemUsed = MemUsedCGroup sysutil.RegisterGetMemoryCapacity(MemTotalCGroup) - log.Info("use cgroup memory hook", zap.Int64("cgroupMemorySize", int64(cgroupValue)), zap.Int64("physicalMemorySize", int64(physicalValue))) + log.Info("use cgroup memory hook", zap.Int64("cgroup-memory-size", int64(cgroupValue)), zap.Int64("physical-memory-size", int64(physicalValue))) } else { - log.Info("use physical memory hook", zap.Int64("cgroupMemorySize", int64(cgroupValue)), zap.Int64("physicalMemorySize", int64(physicalValue))) + log.Info("use physical memory hook", zap.Int64("cgroup-memory-size", int64(cgroupValue)), zap.Int64("physical-memory-size", int64(physicalValue))) } _, err = MemTotal() mustNil(err) diff --git a/pkg/schedule/config/store_config.go b/pkg/schedule/config/store_config.go index 5575f0d9d56..cbf085d93fb 100644 --- a/pkg/schedule/config/store_config.go +++ b/pkg/schedule/config/store_config.go @@ -190,7 +190,7 @@ func (c *StoreConfig) CheckRegionKeys(keys, mergeKeys uint64) error { } if smallKeys := keys % c.GetRegionSplitKeys(); smallKeys <= mergeKeys && smallKeys > 0 { - log.Debug("region keys is too small", zap.Uint64("keys", keys), zap.Uint64("merge-keys", mergeKeys), zap.Uint64("smallSize", smallKeys)) + log.Debug("region keys is too small", zap.Uint64("keys", keys), zap.Uint64("merge-keys", mergeKeys), zap.Uint64("small-keys", smallKeys)) return errs.ErrCheckerMergeAgain.FastGenByArgs("the smallest region of the split regions is less than max-merge-region-keys") } return nil diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 79be126f1d4..005e6b4182a 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -262,7 +262,7 @@ func (s *grantHotRegionScheduler) randomSchedule(cluster sche.SchedulerCluster, op, err := s.transfer(cluster, peer.RegionID, srcStoreID, isLeader) if err != nil { log.Debug("fail to create grant hot region operator", zap.Uint64("region-id", peer.RegionID), - zap.Uint64("src store id", srcStoreID), errs.ZapError(err)) + zap.Uint64("src-store-id", srcStoreID), errs.ZapError(err)) continue } return []*operator.Operator{op} diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index d6bf27878a7..553e0b0effd 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -212,7 +212,7 @@ func (gta *GlobalTSOAllocator) primaryElectionLoop() { zap.String("server-name", gta.member.Name()), zap.String("expected-primary-id", expectedPrimary), zap.Uint64("member-id", gta.member.ID()), - zap.String("cur-memberValue", gta.member.MemberValue())) + zap.String("cur-member-value", gta.member.MemberValue())) time.Sleep(200 * time.Millisecond) continue } diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index d22d284e1be..149c68029be 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -1297,7 +1297,7 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget log.Info("start to merge the keyspace group", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList)) + zap.Uint32s("merge-list", mergeList)) defer logutil.LogPanic() defer kgm.wg.Done() @@ -1316,7 +1316,7 @@ mergeLoop: log.Info("merging checker is closed", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList)) + zap.Uint32s("merge-list", mergeList)) return case <-checkTicker.C: } @@ -1326,7 +1326,7 @@ mergeLoop: log.Warn("unable to get the merge target allocator manager", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("keyspace-group-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Error(err)) continue } @@ -1336,7 +1336,7 @@ mergeLoop: log.Debug("current tso node is not the merge target primary", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList)) + zap.Uint32s("merge-list", mergeList)) continue } // Check if the keyspace group primaries in the merge map are all gone. @@ -1351,7 +1351,7 @@ mergeLoop: log.Error("failed to check if the keyspace group primary in the merge list has gone", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Uint32("merge-id", id), zap.Any("remaining", mergeMap), zap.Error(err)) @@ -1370,7 +1370,7 @@ mergeLoop: "start to calculate the newly merged TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList)) + zap.Uint32s("merge-list", mergeList)) // All the keyspace group primaries in the merge list are gone, // calculate the newly merged TSO to make sure it is greater than the original ones. var mergedTS time.Time @@ -1380,7 +1380,7 @@ mergeLoop: log.Error("failed to load the keyspace group TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Uint32("merge-id", id), zap.Time("ts", ts), zap.Error(err)) @@ -1396,7 +1396,7 @@ mergeLoop: log.Info("start to set the newly merged TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Time("merged-ts", mergedTS)) err = am.GetAllocator().SetTSO( tsoutil.GenerateTS(tsoutil.GenerateTimestamp(mergedTS, 1)), @@ -1405,7 +1405,7 @@ mergeLoop: log.Error("failed to update the newly merged TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Time("merged-ts", mergedTS), zap.Error(err)) continue @@ -1417,7 +1417,7 @@ mergeLoop: log.Error("failed to finish the merge", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Error(err)) continue } @@ -1425,7 +1425,7 @@ mergeLoop: log.Info("finished merging keyspace group", zap.String("member", kgm.tsoServiceID.ServiceAddr), zap.Uint32("merge-target-id", mergeTargetID), - zap.Any("merge-list", mergeList), + zap.Uint32s("merge-list", mergeList), zap.Time("merged-ts", mergedTS)) return } diff --git a/server/grpc_service.go b/server/grpc_service.go index 398325cd30a..8db79d3b8f5 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -2803,7 +2803,7 @@ func (s *GrpcServer) ReportMinResolvedTS(ctx context.Context, request *pdpb.Repo } log.Debug("updated min resolved-ts", zap.Uint64("store", storeID), - zap.Uint64("min resolved-ts", minResolvedTS)) + zap.Uint64("min-resolved-ts", minResolvedTS)) return &pdpb.ReportMinResolvedTsResponse{ Header: wrapHeader(), }, nil diff --git a/server/server.go b/server/server.go index 3f397da4d0b..4cae86b6587 100644 --- a/server/server.go +++ b/server/server.go @@ -2042,7 +2042,7 @@ func (s *Server) GetExternalTS() uint64 { func (s *Server) SetExternalTS(externalTS, globalTS uint64) error { if tsoutil.CompareTimestampUint64(externalTS, globalTS) == 1 { desc := "the external timestamp should not be larger than global ts" - log.Error(desc, zap.Uint64("request timestamp", externalTS), zap.Uint64("global ts", globalTS)) + log.Error(desc, zap.Uint64("request-timestamp", externalTS), zap.Uint64("global-ts", globalTS)) return errors.New(desc) } c := s.GetRaftCluster() diff --git a/server/util.go b/server/util.go index 1764e4e9850..2f05c06b8f5 100644 --- a/server/util.go +++ b/server/util.go @@ -54,7 +54,7 @@ func CheckPDVersionWithClusterVersion(opt *config.PersistOptions) { if pdVersion.LessThan(clusterVersion) { log.Warn( "PD version less than cluster version, please upgrade PD", - zap.String("PD-version", pdVersion.String()), + zap.String("pd-version", pdVersion.String()), zap.String("cluster-version", clusterVersion.String())) } } From 0bfa31f9697dd7ce5a495f235eeb9eedaca67921 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 2 Jan 2025 15:23:21 +0800 Subject: [PATCH 27/33] cluster: fix panic when minResolvedTS is not initialized (#8965) close tikv/pd#8964 Signed-off-by: Ryan Leung --- server/cluster/cluster.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 699b43e7901..d2f3855d14e 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -2283,7 +2283,8 @@ func (c *RaftCluster) CheckAndUpdateMinResolvedTS() (uint64, bool) { newMinResolvedTS = s.GetMinResolvedTS() } } - oldMinResolvedTS := c.minResolvedTS.Load().(uint64) + // Avoid panic when minResolvedTS is not initialized. + oldMinResolvedTS, _ := c.minResolvedTS.Load().(uint64) if newMinResolvedTS == math.MaxUint64 || newMinResolvedTS <= oldMinResolvedTS { return oldMinResolvedTS, false } From 7a30ebc972ffa2f0f76e6f99f9c55a68f975669e Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Thu, 2 Jan 2025 15:35:42 +0800 Subject: [PATCH 28/33] server: advance ServerStart check (#8951) close tikv/pd#8950 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/clients/tso/client.go | 3 +++ client/go.mod | 2 +- client/go.sum | 8 ++------ go.mod | 2 +- go.sum | 5 ++--- server/grpc_service.go | 15 ++++++++++---- server/server.go | 1 + tests/integrations/client/client_test.go | 26 ++++++++++++++++++++++++ tests/integrations/go.mod | 2 +- tests/integrations/go.sum | 7 ++----- tools/go.mod | 2 +- tools/go.sum | 7 ++----- 12 files changed, 53 insertions(+), 27 deletions(-) diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index c6caa8b985f..d24dba52394 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -281,6 +281,9 @@ func (c *Cli) connectionCtxsUpdater() { // Because the TSO Follower Proxy is enabled, // the periodic check needs to be performed. setNewUpdateTicker(sd.MemberUpdateInterval) + failpoint.Inject("speedUpTsoDispatcherUpdateInterval", func() { + setNewUpdateTicker(10 * time.Millisecond) + }) } else if !enableTSOFollowerProxy && updateTicker.C != nil { // Because the TSO Follower Proxy is disabled, // the periodic check needs to be turned off. diff --git a/client/go.mod b/client/go.mod index b26abcbea55..78aef084ff7 100644 --- a/client/go.mod +++ b/client/go.mod @@ -9,7 +9,7 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/opentracing/opentracing-go v1.2.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.20.5 diff --git a/client/go.sum b/client/go.sum index 36c58efb823..4cca5ba3ad5 100644 --- a/client/go.sum +++ b/client/go.sum @@ -45,11 +45,10 @@ github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1y github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 h1:xYNSJjYNur4Dr5bV+9BXK9n5E0T1zlcAN25XX68+mOg= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= @@ -69,7 +68,6 @@ github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0leargg github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 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= @@ -158,7 +156,6 @@ google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6h google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= @@ -167,7 +164,6 @@ gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYs gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= diff --git a/go.mod b/go.mod index 9c8a7bc90a5..c1107b6ffc5 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d github.com/pingcap/errcode v0.3.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 diff --git a/go.sum b/go.sum index a2f070397d1..e6156e9ecf7 100644 --- a/go.sum +++ b/go.sum @@ -386,12 +386,11 @@ github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JH github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 h1:xYNSJjYNur4Dr5bV+9BXK9n5E0T1zlcAN25XX68+mOg= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= diff --git a/server/grpc_service.go b/server/grpc_service.go index 8db79d3b8f5..d3fc5c58d7f 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -325,6 +325,9 @@ func (s *GrpcServer) GetMinTS( // GetMinTSFromTSOService queries all tso servers and gets the minimum timestamp across // all keyspace groups. func (s *GrpcServer) GetMinTSFromTSOService() (*pdpb.Timestamp, error) { + if s.IsClosed() { + return nil, errs.ErrNotStarted + } addrs := s.keyspaceGroupManager.GetTSOServiceAddrs() if len(addrs) == 0 { return &pdpb.Timestamp{}, errs.ErrGetMinTS.FastGenByArgs("no tso servers/pods discovered") @@ -536,6 +539,11 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { return errors.WithStack(err) } + // TSO uses leader lease to determine validity. No need to check leader here. + if s.IsClosed() { + return errs.ErrNotStarted + } + forwardedHost := grpcutil.GetForwardedHost(stream.Context()) if !s.isLocalRequest(forwardedHost) { clientConn, err := s.getDelegateClient(s.ctx, forwardedHost) @@ -570,10 +578,6 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { } start := time.Now() - // TSO uses leader lease to determine validity. No need to check leader here. - if s.IsClosed() { - return errs.ErrNotStarted - } if clusterID := keypath.ClusterID(); request.GetHeader().GetClusterId() != clusterID { return errs.ErrMismatchClusterID(clusterID, request.GetHeader().GetClusterId()) } @@ -710,6 +714,9 @@ func (s *GrpcServer) IsSnapshotRecovering(ctx context.Context, _ *pdpb.IsSnapsho return nil, errs.ErrGRPCRateLimitExceeded(err) } } + if s.IsClosed() { + return nil, errs.ErrNotStarted + } // recovering mark is stored in etcd directly, there's no need to forward. marked, err := s.Server.IsSnapshotRecovering(ctx) if err != nil { diff --git a/server/server.go b/server/server.go index 4cae86b6587..94250128fe3 100644 --- a/server/server.go +++ b/server/server.go @@ -506,6 +506,7 @@ func (s *Server) startServer(ctx context.Context) error { s.grpcServiceRateLimiter.Update(service, ratelimit.InitLimiter()) } + failpoint.InjectCall("delayStartServer") // Server has started. atomic.StoreInt64(&s.isRunning, 1) bs.ServerMaxProcsGauge.Set(float64(runtime.GOMAXPROCS(0))) diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index fadfb952e4c..2018860130e 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -270,6 +270,7 @@ func TestTSOFollowerProxy(t *testing.T) { defer cli1.Close() cli2 := setupCli(ctx, re, endpoints) defer cli2.Close() + re.NoError(failpoint.Enable("github.com/tikv/pd/client/clients/tso/speedUpTsoDispatcherUpdateInterval", "return(true)")) err = cli2.UpdateOption(opt.EnableTSOFollowerProxy, true) re.NoError(err) @@ -296,6 +297,31 @@ func TestTSOFollowerProxy(t *testing.T) { } wg.Wait() + followerServer := cluster.GetServer(cluster.GetFollower()) + re.NoError(followerServer.Stop()) + ch := make(chan struct{}) + re.NoError(failpoint.EnableCall("github.com/tikv/pd/server/delayStartServer", func() { + // Server is not in `Running` state, so the follower proxy should return + // error while create stream. + ch <- struct{}{} + })) + wg.Add(1) + go func() { + defer wg.Done() + re.NoError(followerServer.Run()) + }() + re.Eventually(func() bool { + _, _, err := cli2.GetTS(context.Background()) + if err == nil { + return false + } + return strings.Contains(err.Error(), "server not started") + }, 3*time.Second, 10*time.Millisecond) + <-ch + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServer")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/clients/tso/speedUpTsoDispatcherUpdateInterval")) + wg.Wait() + // Disable the follower proxy and check if the stream is updated. err = cli2.UpdateOption(opt.EnableTSOFollowerProxy, false) re.NoError(err) diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index c5fcb617014..ec1d74923d6 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -13,7 +13,7 @@ require ( github.com/docker/go-units v0.5.0 github.com/go-sql-driver/mysql v1.7.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.20.5 diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index 5a48549ef65..5f78324c3c2 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -379,12 +379,11 @@ github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JH github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 h1:xYNSJjYNur4Dr5bV+9BXK9n5E0T1zlcAN25XX68+mOg= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= @@ -429,7 +428,6 @@ github.com/samber/lo v1.37.0 h1:XjVcB8g6tgUp8rsPsJ2CvhClfImrpL04YpQHXeHPhRw= github.com/samber/lo v1.37.0/go.mod h1:9vaz2O4o8oOnK23pd2TrXufcbdbJIa3b6cstBWKpopA= github.com/sasha-s/go-deadlock v0.3.5 h1:tNCOEEDG6tBqrNDOX35j/7hL5FcFViG6awUGROb2NsU= github.com/sasha-s/go-deadlock v0.3.5/go.mod h1:bugP6EGbdGYObIlx7pUZtWqlvo8k9H6vCBBsiChJQ5U= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/gopsutil/v3 v3.23.3 h1:Syt5vVZXUDXPEXpIBt5ziWsJ4LdSAAxF4l/xZeQgSEE= github.com/shirou/gopsutil/v3 v3.23.3/go.mod h1:lSBNN6t3+D6W5e5nXTxc8KIMMVxAcS+6IJlffjRRlMU= @@ -748,7 +746,6 @@ google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6h google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/tools/go.mod b/tools/go.mod index ded2e2e82c8..31309986d92 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -21,7 +21,7 @@ require ( github.com/influxdata/tdigest v0.0.1 github.com/mattn/go-shellwords v1.0.12 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 + github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.20.5 diff --git a/tools/go.sum b/tools/go.sum index bd68a4f0ca1..b9c49d466ef 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -380,12 +380,11 @@ github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JH github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= -github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 h1:tdMsjOqUR7YXHoBitzdebTvOjs/swniBTOLy5XiMtuE= +github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86/go.mod h1:exzhVYca3WRtd6gclGNErRWb1qEgff3LYta0LvRmON4= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037 h1:xYNSJjYNur4Dr5bV+9BXK9n5E0T1zlcAN25XX68+mOg= github.com/pingcap/kvproto v0.0.0-20241120071417-b5b7843d9037/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= @@ -432,7 +431,6 @@ github.com/samber/lo v1.37.0 h1:XjVcB8g6tgUp8rsPsJ2CvhClfImrpL04YpQHXeHPhRw= github.com/samber/lo v1.37.0/go.mod h1:9vaz2O4o8oOnK23pd2TrXufcbdbJIa3b6cstBWKpopA= github.com/sasha-s/go-deadlock v0.3.5 h1:tNCOEEDG6tBqrNDOX35j/7hL5FcFViG6awUGROb2NsU= github.com/sasha-s/go-deadlock v0.3.5/go.mod h1:bugP6EGbdGYObIlx7pUZtWqlvo8k9H6vCBBsiChJQ5U= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= github.com/shirou/gopsutil/v3 v3.23.3 h1:Syt5vVZXUDXPEXpIBt5ziWsJ4LdSAAxF4l/xZeQgSEE= github.com/shirou/gopsutil/v3 v3.23.3/go.mod h1:lSBNN6t3+D6W5e5nXTxc8KIMMVxAcS+6IJlffjRRlMU= @@ -759,7 +757,6 @@ google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6h google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= 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= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= From 41919ad57acca17c6e2c24a19a1c0185c355e469 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Thu, 2 Jan 2025 16:46:51 +0800 Subject: [PATCH 29/33] test: make TestPreparingProgress stable (#8966) close tikv/pd#8693 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- tests/server/api/api_test.go | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/tests/server/api/api_test.go b/tests/server/api/api_test.go index faa22ce08f4..44a0ae69a46 100644 --- a/tests/server/api/api_test.go +++ b/tests/server/api/api_test.go @@ -1092,13 +1092,24 @@ func TestPreparingProgress(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs")) } -func sendRequest(re *require.Assertions, url string, method string, statusCode int) []byte { +func sendRequest(re *require.Assertions, url string, method string, statusCode int) (output []byte) { req, _ := http.NewRequest(method, url, http.NoBody) - resp, err := tests.TestDialClient.Do(req) - re.NoError(err) - re.Equal(statusCode, resp.StatusCode) - output, err := io.ReadAll(resp.Body) - re.NoError(err) - resp.Body.Close() + + testutil.Eventually(re, func() bool { + resp, err := tests.TestDialClient.Do(req) + re.NoError(err) + defer resp.Body.Close() + + // Due to service unavailability caused by environmental issues, + // we will retry it. + if resp.StatusCode == http.StatusServiceUnavailable { + return false + } + re.Equal(statusCode, resp.StatusCode) + output, err = io.ReadAll(resp.Body) + re.NoError(err) + return true + }) + return output } From 6a0ed869ce3d233a02c3c6ecff642546dca5d324 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Mon, 6 Jan 2025 13:53:22 +0800 Subject: [PATCH 30/33] client: retry checkServiceModeChanged to make tso client work normal (#8963) close tikv/pd#8962 Signed-off-by: okJiang <819421878@qq.com> Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/servicediscovery/pd_service_discovery.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/client/servicediscovery/pd_service_discovery.go b/client/servicediscovery/pd_service_discovery.go index 5530f3cfa9b..931f950c6d1 100644 --- a/client/servicediscovery/pd_service_discovery.go +++ b/client/servicediscovery/pd_service_discovery.go @@ -509,8 +509,9 @@ func (c *pdServiceDiscovery) Init() error { } } - if err := c.checkServiceModeChanged(); err != nil { - log.Warn("[pd] failed to check service mode and will check later", zap.Error(err)) + if err := c.initRetry(c.checkServiceModeChanged); err != nil { + c.cancel() + return err } c.wg.Add(3) From 5c4ab57d68dea7c82b511c06f8ccfa1e075889b2 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 7 Jan 2025 11:26:58 +0800 Subject: [PATCH 31/33] client: use interceptor for circuit breaker (#8936) ref tikv/pd#8678 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 28 ++------- client/inner_client.go | 22 ++----- client/opt/option.go | 27 ++------ client/pkg/circuitbreaker/circuit_breaker.go | 60 ++++++++++++------ .../circuitbreaker/circuit_breaker_test.go | 63 +++++++++---------- client/pkg/utils/grpcutil/grpcutil.go | 40 +++++++++++- tests/integrations/client/client_test.go | 63 ++++++++++--------- 7 files changed, 156 insertions(+), 147 deletions(-) diff --git a/client/client.go b/client/client.go index 2fc9bd3ef0d..7aa28cbc0cd 100644 --- a/client/client.go +++ b/client/client.go @@ -25,8 +25,6 @@ import ( "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -42,7 +40,6 @@ import ( "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" "github.com/tikv/pd/client/pkg/caller" - cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/utils/tlsutil" sd "github.com/tikv/pd/client/servicediscovery" ) @@ -460,12 +457,6 @@ func (c *client) UpdateOption(option opt.DynamicOption, value any) error { return errors.New("[pd] invalid value type for TSOClientRPCConcurrency option, it should be int") } c.inner.option.SetTSOClientRPCConcurrency(value) - case opt.RegionMetadataCircuitBreakerSettings: - applySettingsChange, ok := value.(func(config *cb.Settings)) - if !ok { - return errors.New("[pd] invalid value type for RegionMetadataCircuitBreakerSettings option, it should be pd.Settings") - } - c.inner.regionMetaCircuitBreaker.ChangeSettings(applySettingsChange) default: return errors.New("[pd] unsupported client option") } @@ -660,13 +651,7 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...opt.GetRegio if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { - region, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) - failpoint.Inject("triggerCircuitBreaker", func() { - err = status.Error(codes.ResourceExhausted, "resource exhausted") - }) - return region, isOverloaded(err), err - }) + resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegion(cctx, req) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { @@ -706,10 +691,7 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...opt.GetR if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { - resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetPrevRegion(cctx, req) - return resp, isOverloaded(err), err - }) + resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetPrevRegion(cctx, req) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { @@ -749,10 +731,8 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...opt if serviceClient == nil { return nil, errs.ErrClientGetProtoClient } - resp, err := c.inner.regionMetaCircuitBreaker.Execute(func() (*pdpb.GetRegionResponse, cb.Overloading, error) { - resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegionByID(cctx, req) - return resp, isOverloaded(err), err - }) + + resp, err := pdpb.NewPDClient(serviceClient.GetClientConn()).GetRegionByID(cctx, req) if serviceClient.NeedRetry(resp.GetHeader().GetError(), err) { protoClient, cctx := c.getClientAndContext(ctx) if protoClient == nil { diff --git a/client/inner_client.go b/client/inner_client.go index 91f999dd3b5..404cbcf0b80 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -8,8 +8,6 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/pdpb" @@ -19,7 +17,6 @@ import ( "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/metrics" "github.com/tikv/pd/client/opt" - cb "github.com/tikv/pd/client/pkg/circuitbreaker" sd "github.com/tikv/pd/client/servicediscovery" ) @@ -29,11 +26,10 @@ const ( ) type innerClient struct { - keyspaceID uint32 - svrUrls []string - pdSvcDiscovery sd.ServiceDiscovery - tokenDispatcher *tokenDispatcher - regionMetaCircuitBreaker *cb.CircuitBreaker[*pdpb.GetRegionResponse] + keyspaceID uint32 + svrUrls []string + pdSvcDiscovery sd.ServiceDiscovery + tokenDispatcher *tokenDispatcher // For service mode switching. serviceModeKeeper @@ -59,7 +55,6 @@ func (c *innerClient) init(updateKeyspaceIDCb sd.UpdateKeyspaceIDFunc) error { } return err } - c.regionMetaCircuitBreaker = cb.NewCircuitBreaker[*pdpb.GetRegionResponse]("region_meta", c.option.RegionMetaCircuitBreakerSettings) return nil } @@ -252,12 +247,3 @@ func (c *innerClient) dispatchTSORequestWithRetry(ctx context.Context) tso.TSFut } return req } - -func isOverloaded(err error) cb.Overloading { - switch status.Code(errors.Cause(err)) { - case codes.DeadlineExceeded, codes.Unavailable, codes.ResourceExhausted: - return cb.Yes - default: - return cb.No - } -} diff --git a/client/opt/option.go b/client/opt/option.go index af95a225fab..2aa9be8ae7f 100644 --- a/client/opt/option.go +++ b/client/opt/option.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/errors" - cb "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" ) @@ -50,8 +49,6 @@ const ( EnableFollowerHandle // TSOClientRPCConcurrency controls the amount of ongoing TSO RPC requests at the same time in a single TSO client. TSOClientRPCConcurrency - // RegionMetadataCircuitBreakerSettings controls settings for circuit breaker for region metadata requests. - RegionMetadataCircuitBreakerSettings dynamicOptionCount ) @@ -72,18 +69,16 @@ type Option struct { // Dynamic options. dynamicOptions [dynamicOptionCount]atomic.Value - EnableTSOFollowerProxyCh chan struct{} - RegionMetaCircuitBreakerSettings cb.Settings + EnableTSOFollowerProxyCh chan struct{} } // NewOption creates a new PD client option with the default values set. func NewOption() *Option { co := &Option{ - Timeout: defaultPDTimeout, - MaxRetryTimes: maxInitClusterRetries, - EnableTSOFollowerProxyCh: make(chan struct{}, 1), - InitMetrics: true, - RegionMetaCircuitBreakerSettings: cb.AlwaysClosedSettings, + Timeout: defaultPDTimeout, + MaxRetryTimes: maxInitClusterRetries, + EnableTSOFollowerProxyCh: make(chan struct{}, 1), + InitMetrics: true, } co.dynamicOptions[MaxTSOBatchWaitInterval].Store(defaultMaxTSOBatchWaitInterval) @@ -154,11 +149,6 @@ func (o *Option) GetTSOClientRPCConcurrency() int { return o.dynamicOptions[TSOClientRPCConcurrency].Load().(int) } -// GetRegionMetadataCircuitBreakerSettings gets circuit breaker settings for PD region metadata calls. -func (o *Option) GetRegionMetadataCircuitBreakerSettings() cb.Settings { - return o.dynamicOptions[RegionMetadataCircuitBreakerSettings].Load().(cb.Settings) -} - // ClientOption configures client. type ClientOption func(*Option) @@ -213,13 +203,6 @@ func WithInitMetricsOption(initMetrics bool) ClientOption { } } -// WithRegionMetaCircuitBreaker configures the client with circuit breaker for region meta calls -func WithRegionMetaCircuitBreaker(config cb.Settings) ClientOption { - return func(op *Option) { - op.RegionMetaCircuitBreakerSettings = config - } -} - // WithBackoffer configures the client with backoffer. func WithBackoffer(bo *retry.Backoffer) ClientOption { return func(op *Option) { diff --git a/client/pkg/circuitbreaker/circuit_breaker.go b/client/pkg/circuitbreaker/circuit_breaker.go index 2c65f4f1965..0acee5d5c8d 100644 --- a/client/pkg/circuitbreaker/circuit_breaker.go +++ b/client/pkg/circuitbreaker/circuit_breaker.go @@ -14,6 +14,7 @@ package circuitbreaker import ( + "context" "fmt" "strings" "sync" @@ -62,12 +63,12 @@ var AlwaysClosedSettings = Settings{ } // CircuitBreaker is a state machine to prevent sending requests that are likely to fail. -type CircuitBreaker[T any] struct { +type CircuitBreaker struct { config *Settings name string mutex sync.Mutex - state *State[T] + state *State successCounter prometheus.Counter errorCounter prometheus.Counter @@ -102,8 +103,8 @@ func (s StateType) String() string { var replacer = strings.NewReplacer(" ", "_", "-", "_") // NewCircuitBreaker returns a new CircuitBreaker configured with the given Settings. -func NewCircuitBreaker[T any](name string, st Settings) *CircuitBreaker[T] { - cb := new(CircuitBreaker[T]) +func NewCircuitBreaker(name string, st Settings) *CircuitBreaker { + cb := new(CircuitBreaker) cb.name = name cb.config = &st cb.state = cb.newState(time.Now(), StateClosed) @@ -118,7 +119,7 @@ func NewCircuitBreaker[T any](name string, st Settings) *CircuitBreaker[T] { // ChangeSettings changes the CircuitBreaker settings. // The changes will be reflected only in the next evaluation window. -func (cb *CircuitBreaker[T]) ChangeSettings(apply func(config *Settings)) { +func (cb *CircuitBreaker) ChangeSettings(apply func(config *Settings)) { cb.mutex.Lock() defer cb.mutex.Unlock() @@ -129,12 +130,11 @@ func (cb *CircuitBreaker[T]) ChangeSettings(apply func(config *Settings)) { // Execute calls the given function if the CircuitBreaker is closed and returns the result of execution. // Execute returns an error instantly if the CircuitBreaker is open. // https://github.com/tikv/rfcs/blob/master/text/0115-circuit-breaker.md -func (cb *CircuitBreaker[T]) Execute(call func() (T, Overloading, error)) (T, error) { +func (cb *CircuitBreaker) Execute(call func() (Overloading, error)) error { state, err := cb.onRequest() if err != nil { cb.fastFailCounter.Inc() - var defaultValue T - return defaultValue, err + return err } defer func() { @@ -146,13 +146,13 @@ func (cb *CircuitBreaker[T]) Execute(call func() (T, Overloading, error)) (T, er } }() - result, overloaded, err := call() + overloaded, err := call() cb.emitMetric(overloaded, err) cb.onResult(state, overloaded) - return result, err + return err } -func (cb *CircuitBreaker[T]) onRequest() (*State[T], error) { +func (cb *CircuitBreaker) onRequest() (*State, error) { cb.mutex.Lock() defer cb.mutex.Unlock() @@ -161,7 +161,7 @@ func (cb *CircuitBreaker[T]) onRequest() (*State[T], error) { return state, err } -func (cb *CircuitBreaker[T]) onResult(state *State[T], overloaded Overloading) { +func (cb *CircuitBreaker) onResult(state *State, overloaded Overloading) { cb.mutex.Lock() defer cb.mutex.Unlock() @@ -170,7 +170,7 @@ func (cb *CircuitBreaker[T]) onResult(state *State[T], overloaded Overloading) { state.onResult(overloaded) } -func (cb *CircuitBreaker[T]) emitMetric(overloaded Overloading, err error) { +func (cb *CircuitBreaker) emitMetric(overloaded Overloading, err error) { switch overloaded { case No: cb.successCounter.Inc() @@ -185,9 +185,9 @@ func (cb *CircuitBreaker[T]) emitMetric(overloaded Overloading, err error) { } // State represents the state of CircuitBreaker. -type State[T any] struct { +type State struct { stateType StateType - cb *CircuitBreaker[T] + cb *CircuitBreaker end time.Time pendingCount uint32 @@ -196,7 +196,7 @@ type State[T any] struct { } // newState creates a new State with the given configuration and reset all success/failure counters. -func (cb *CircuitBreaker[T]) newState(now time.Time, stateType StateType) *State[T] { +func (cb *CircuitBreaker) newState(now time.Time, stateType StateType) *State { var end time.Time var pendingCount uint32 switch stateType { @@ -211,7 +211,7 @@ func (cb *CircuitBreaker[T]) newState(now time.Time, stateType StateType) *State default: panic("unknown state") } - return &State[T]{ + return &State{ cb: cb, stateType: stateType, pendingCount: pendingCount, @@ -227,7 +227,7 @@ func (cb *CircuitBreaker[T]) newState(now time.Time, stateType StateType) *State // Open state fails all request, it has a fixed duration of `Settings.CoolDownInterval` and always moves to HalfOpen state at the end of the interval. // HalfOpen state does not have a fixed duration and lasts till `Settings.HalfOpenSuccessCount` are evaluated. // If any of `Settings.HalfOpenSuccessCount` fails then it moves back to Open state, otherwise it moves to Closed state. -func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { +func (s *State) onRequest(cb *CircuitBreaker) (*State, error) { var now = time.Now() switch s.stateType { case StateClosed: @@ -299,7 +299,7 @@ func (s *State[T]) onRequest(cb *CircuitBreaker[T]) (*State[T], error) { } } -func (s *State[T]) onResult(overloaded Overloading) { +func (s *State) onResult(overloaded Overloading) { switch overloaded { case No: s.successCount++ @@ -309,3 +309,25 @@ func (s *State[T]) onResult(overloaded Overloading) { panic("unknown state") } } + +// Define context key type +type cbCtxKey struct{} + +// Key used to store circuit breaker +var CircuitBreakerKey = cbCtxKey{} + +// FromContext retrieves the circuit breaker from the context +func FromContext(ctx context.Context) *CircuitBreaker { + if ctx == nil { + return nil + } + if cb, ok := ctx.Value(CircuitBreakerKey).(*CircuitBreaker); ok { + return cb + } + return nil +} + +// WithCircuitBreaker stores the circuit breaker into a new context +func WithCircuitBreaker(ctx context.Context, cb *CircuitBreaker) context.Context { + return context.WithValue(ctx, CircuitBreakerKey, cb) +} diff --git a/client/pkg/circuitbreaker/circuit_breaker_test.go b/client/pkg/circuitbreaker/circuit_breaker_test.go index 07a3c06f86e..e62e55c1ab8 100644 --- a/client/pkg/circuitbreaker/circuit_breaker_test.go +++ b/client/pkg/circuitbreaker/circuit_breaker_test.go @@ -24,7 +24,7 @@ import ( ) // advance emulate the state machine clock moves forward by the given duration -func (cb *CircuitBreaker[T]) advance(duration time.Duration) { +func (cb *CircuitBreaker) advance(duration time.Duration) { cb.state.end = cb.state.end.Add(-duration - 1) } @@ -40,26 +40,24 @@ var minCountToOpen = int(settings.MinQPSForOpen * uint32(settings.ErrorRateWindo func TestCircuitBreakerExecuteWrapperReturnValues(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) originalError := errors.New("circuit breaker is open") - result, err := cb.Execute(func() (int, Overloading, error) { - return 42, No, originalError + err := cb.Execute(func() (Overloading, error) { + return No, originalError }) re.Equal(err, originalError) - re.Equal(42, result) // same by interpret the result as overloading error - result, err = cb.Execute(func() (int, Overloading, error) { - return 42, Yes, originalError + err = cb.Execute(func() (Overloading, error) { + return Yes, originalError }) re.Equal(err, originalError) - re.Equal(42, result) } func TestCircuitBreakerOpenState(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) driveQPS(cb, minCountToOpen, Yes, re) re.Equal(StateClosed, cb.state.stateType) assertSucceeds(cb, re) // no error till ErrorRateWindow is finished @@ -70,7 +68,7 @@ func TestCircuitBreakerOpenState(t *testing.T) { func TestCircuitBreakerCloseStateNotEnoughQPS(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) driveQPS(cb, minCountToOpen/2, Yes, re) cb.advance(settings.ErrorRateWindow) @@ -80,7 +78,7 @@ func TestCircuitBreakerCloseStateNotEnoughQPS(t *testing.T) { func TestCircuitBreakerCloseStateNotEnoughErrorRate(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) driveQPS(cb, minCountToOpen/4, Yes, re) driveQPS(cb, minCountToOpen, No, re) @@ -91,7 +89,7 @@ func TestCircuitBreakerCloseStateNotEnoughErrorRate(t *testing.T) { func TestCircuitBreakerHalfOpenToClosed(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) driveQPS(cb, minCountToOpen, Yes, re) cb.advance(settings.ErrorRateWindow) @@ -109,7 +107,7 @@ func TestCircuitBreakerHalfOpenToClosed(t *testing.T) { func TestCircuitBreakerHalfOpenToOpen(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) driveQPS(cb, minCountToOpen, Yes, re) cb.advance(settings.ErrorRateWindow) @@ -118,8 +116,8 @@ func TestCircuitBreakerHalfOpenToOpen(t *testing.T) { cb.advance(settings.CoolDownInterval) assertSucceeds(cb, re) re.Equal(StateHalfOpen, cb.state.stateType) - _, err := cb.Execute(func() (int, Overloading, error) { - return 42, Yes, nil // this trip circuit breaker again + err := cb.Execute(func() (Overloading, error) { + return Yes, nil // this trip circuit breaker again }) re.NoError(err) re.Equal(StateHalfOpen, cb.state.stateType) @@ -149,10 +147,10 @@ func TestCircuitBreakerHalfOpenFailOverPendingCount(t *testing.T) { defer func() { end <- true }() - _, err := cb.Execute(func() (int, Overloading, error) { + err := cb.Execute(func() (Overloading, error) { start <- true <-wait - return 42, No, nil + return No, nil }) re.NoError(err) }() @@ -178,7 +176,7 @@ func TestCircuitBreakerHalfOpenFailOverPendingCount(t *testing.T) { func TestCircuitBreakerCountOnlyRequestsInSameWindow(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", settings) + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) start := make(chan bool) @@ -188,10 +186,10 @@ func TestCircuitBreakerCountOnlyRequestsInSameWindow(t *testing.T) { defer func() { end <- true }() - _, err := cb.Execute(func() (int, Overloading, error) { + err := cb.Execute(func() (Overloading, error) { start <- true <-wait - return 42, No, nil + return No, nil }) re.NoError(err) }() @@ -214,7 +212,7 @@ func TestCircuitBreakerCountOnlyRequestsInSameWindow(t *testing.T) { func TestCircuitBreakerChangeSettings(t *testing.T) { re := require.New(t) - cb := NewCircuitBreaker[int]("test_cb", AlwaysClosedSettings) + cb := NewCircuitBreaker("test_cb", AlwaysClosedSettings) driveQPS(cb, int(AlwaysClosedSettings.MinQPSForOpen*uint32(AlwaysClosedSettings.ErrorRateWindow.Seconds())), Yes, re) cb.advance(AlwaysClosedSettings.ErrorRateWindow) assertSucceeds(cb, re) @@ -231,8 +229,8 @@ func TestCircuitBreakerChangeSettings(t *testing.T) { re.Equal(StateOpen, cb.state.stateType) } -func newCircuitBreakerMovedToHalfOpenState(re *require.Assertions) *CircuitBreaker[int] { - cb := NewCircuitBreaker[int]("test_cb", settings) +func newCircuitBreakerMovedToHalfOpenState(re *require.Assertions) *CircuitBreaker { + cb := NewCircuitBreaker("test_cb", settings) re.Equal(StateClosed, cb.state.stateType) driveQPS(cb, minCountToOpen, Yes, re) cb.advance(settings.ErrorRateWindow) @@ -242,29 +240,28 @@ func newCircuitBreakerMovedToHalfOpenState(re *require.Assertions) *CircuitBreak return cb } -func driveQPS(cb *CircuitBreaker[int], count int, overload Overloading, re *require.Assertions) { +func driveQPS(cb *CircuitBreaker, count int, overload Overloading, re *require.Assertions) { for range count { - _, err := cb.Execute(func() (int, Overloading, error) { - return 42, overload, nil + err := cb.Execute(func() (Overloading, error) { + return overload, nil }) re.NoError(err) } } -func assertFastFail(cb *CircuitBreaker[int], re *require.Assertions) { +func assertFastFail(cb *CircuitBreaker, re *require.Assertions) { var executed = false - _, err := cb.Execute(func() (int, Overloading, error) { + err := cb.Execute(func() (Overloading, error) { executed = true - return 42, No, nil + return No, nil }) re.Equal(err, errs.ErrCircuitBreakerOpen) re.False(executed) } -func assertSucceeds(cb *CircuitBreaker[int], re *require.Assertions) { - result, err := cb.Execute(func() (int, Overloading, error) { - return 42, No, nil +func assertSucceeds(cb *CircuitBreaker, re *require.Assertions) { + err := cb.Execute(func() (Overloading, error) { + return No, nil }) re.NoError(err) - re.Equal(42, result) } diff --git a/client/pkg/utils/grpcutil/grpcutil.go b/client/pkg/utils/grpcutil/grpcutil.go index b73d117fe84..235e1088747 100644 --- a/client/pkg/utils/grpcutil/grpcutil.go +++ b/client/pkg/utils/grpcutil/grpcutil.go @@ -24,15 +24,18 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/backoff" + "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/tikv/pd/client/errs" + "github.com/tikv/pd/client/pkg/circuitbreaker" "github.com/tikv/pd/client/pkg/retry" ) @@ -71,6 +74,36 @@ func UnaryBackofferInterceptor() grpc.UnaryClientInterceptor { } } +// UnaryCircuitBreakerInterceptor is a gRPC interceptor that adds a circuit breaker to the call. +func UnaryCircuitBreakerInterceptor() grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply any, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + cb := circuitbreaker.FromContext(ctx) + if cb == nil { + return invoker(ctx, method, req, reply, cc, opts...) + } + err := cb.Execute(func() (circuitbreaker.Overloading, error) { + err := invoker(ctx, method, req, reply, cc, opts...) + failpoint.Inject("triggerCircuitBreaker", func() { + err = status.Error(codes.ResourceExhausted, "resource exhausted") + }) + return isOverloaded(err), err + }) + if err != nil { + return err + } + return nil + } +} + +func isOverloaded(err error) circuitbreaker.Overloading { + switch status.Code(errors.Cause(err)) { + case codes.DeadlineExceeded, codes.Unavailable, codes.ResourceExhausted: + return circuitbreaker.Yes + default: + return circuitbreaker.No + } +} + // GetClientConn returns a gRPC client connection. // creates a client connection to the given target. By default, it's // a non-blocking dial (the function won't wait for connections to be @@ -96,7 +129,10 @@ func GetClientConn(ctx context.Context, addr string, tlsCfg *tls.Config, do ...g } // Add backoffer interceptor - retryOpt := grpc.WithUnaryInterceptor(UnaryBackofferInterceptor()) + retryOpt := grpc.WithChainUnaryInterceptor(UnaryBackofferInterceptor()) + + // Add circuit breaker interceptor + cbOpt := grpc.WithChainUnaryInterceptor(UnaryCircuitBreakerInterceptor()) // Add retry related connection parameters backoffOpts := grpc.WithConnectParams(grpc.ConnectParams{ @@ -108,7 +144,7 @@ func GetClientConn(ctx context.Context, addr string, tlsCfg *tls.Config, do ...g }, }) - do = append(do, opt, retryOpt, backoffOpts) + do = append(do, opt, retryOpt, cbOpt, backoffOpts) cc, err := grpc.DialContext(ctx, u.Host, do...) if err != nil { return nil, errs.ErrGRPCDial.Wrap(err).GenWithStackByCause() diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 2018860130e..397e1079af3 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -2096,28 +2096,30 @@ func TestCircuitBreaker(t *testing.T) { } endpoints := runServer(re, cluster) - cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + cli := setupCli(ctx, re, endpoints) defer cli.Close() + circuitBreaker := cb.NewCircuitBreaker("region_meta", circuitBreakerSettings) + ctx = cb.WithCircuitBreaker(ctx, circuitBreaker) for range 10 { - region, err := cli.GetRegion(context.TODO(), []byte("a")) + region, err := cli.GetRegion(ctx, []byte("a")) re.NoError(err) re.NotNil(region) } - re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker", "return(true)")) for range 100 { - _, err := cli.GetRegion(context.TODO(), []byte("a")) + _, err := cli.GetRegion(ctx, []byte("a")) re.Error(err) } - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.Error(err) re.Contains(err.Error(), "circuit breaker is open") - re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker")) - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.Error(err) re.Contains(err.Error(), "circuit breaker is open") @@ -2125,7 +2127,7 @@ func TestCircuitBreaker(t *testing.T) { time.Sleep(time.Second) for range 10 { - region, err := cli.GetRegion(context.TODO(), []byte("a")) + region, err := cli.GetRegion(ctx, []byte("a")) re.NoError(err) re.NotNil(region) } @@ -2149,34 +2151,35 @@ func TestCircuitBreakerOpenAndChangeSettings(t *testing.T) { } endpoints := runServer(re, cluster) - cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + cli := setupCli(ctx, re, endpoints) defer cli.Close() + circuitBreaker := cb.NewCircuitBreaker("region_meta", circuitBreakerSettings) + ctx = cb.WithCircuitBreaker(ctx, circuitBreaker) for range 10 { - region, err := cli.GetRegion(context.TODO(), []byte("a")) + region, err := cli.GetRegion(ctx, []byte("a")) re.NoError(err) re.NotNil(region) } - re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker", "return(true)")) for range 100 { - _, err := cli.GetRegion(context.TODO(), []byte("a")) + _, err := cli.GetRegion(ctx, []byte("a")) re.Error(err) } - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.Error(err) re.Contains(err.Error(), "circuit breaker is open") - cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + circuitBreaker.ChangeSettings(func(config *cb.Settings) { *config = cb.AlwaysClosedSettings }) - - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.Error(err) re.Contains(err.Error(), "ResourceExhausted") - re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker")) } func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { @@ -2197,23 +2200,26 @@ func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { } endpoints := runServer(re, cluster) - cli := setupCli(ctx, re, endpoints, opt.WithRegionMetaCircuitBreaker(circuitBreakerSettings)) + + cli := setupCli(ctx, re, endpoints) defer cli.Close() + circuitBreaker := cb.NewCircuitBreaker("region_meta", circuitBreakerSettings) + ctx = cb.WithCircuitBreaker(ctx, circuitBreaker) for range 10 { - region, err := cli.GetRegion(context.TODO(), []byte("a")) + region, err := cli.GetRegion(ctx, []byte("a")) re.NoError(err) re.NotNil(region) } - re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker", "return(true)")) for range 100 { - _, err := cli.GetRegion(context.TODO(), []byte("a")) + _, err := cli.GetRegion(ctx, []byte("a")) re.Error(err) } - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.Error(err) re.Contains(err.Error(), "circuit breaker is open") @@ -2221,9 +2227,9 @@ func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { defer os.RemoveAll(fname) // wait for cooldown time.Sleep(time.Second) - re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker")) // trigger circuit breaker state to be half open - _, err = cli.GetRegion(context.TODO(), []byte("a")) + _, err = cli.GetRegion(ctx, []byte("a")) re.NoError(err) testutil.Eventually(re, func() bool { b, _ := os.ReadFile(fname) @@ -2233,17 +2239,16 @@ func TestCircuitBreakerHalfOpenAndChangeSettings(t *testing.T) { }) // The state is half open - re.NoError(failpoint.Enable("github.com/tikv/pd/client/triggerCircuitBreaker", "return(true)")) + re.NoError(failpoint.Enable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker", "return(true)")) // change settings to always closed - cli.UpdateOption(opt.RegionMetadataCircuitBreakerSettings, func(config *cb.Settings) { + circuitBreaker.ChangeSettings(func(config *cb.Settings) { *config = cb.AlwaysClosedSettings }) - // It won't be changed to open state. for range 100 { - _, err := cli.GetRegion(context.TODO(), []byte("a")) + _, err := cli.GetRegion(ctx, []byte("a")) re.Error(err) re.NotContains(err.Error(), "circuit breaker is open") } - re.NoError(failpoint.Disable("github.com/tikv/pd/client/triggerCircuitBreaker")) + re.NoError(failpoint.Disable("github.com/tikv/pd/client/pkg/utils/grpcutil/triggerCircuitBreaker")) } From 973234dd090b342b04dde39c8b43c5160e67d5a4 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 7 Jan 2025 16:12:28 +0800 Subject: [PATCH 32/33] *: remove api mode concept (#8952) ref tikv/pd#8477 Signed-off-by: Ryan Leung --- client/client.go | 26 +-- client/clients/tso/client.go | 2 +- client/clients/tso/dispatcher_test.go | 2 +- client/http/client.go | 10 +- client/inner_client.go | 36 ++-- client/keyspace_client.go | 8 +- client/meta_storage_client.go | 8 +- client/resource_manager_client.go | 4 +- ...discovery.go => mock_service_discovery.go} | 50 ++--- ...vice_discovery.go => service_discovery.go} | 204 +++++++++--------- ...very_test.go => service_discovery_test.go} | 20 +- .../servicediscovery/tso_service_discovery.go | 26 +-- cmd/pd-server/main.go | 14 +- pkg/mcs/registry/registry.go | 8 +- pkg/mcs/scheduling/server/cluster.go | 18 +- pkg/mcs/scheduling/server/config/config.go | 6 +- pkg/mcs/scheduling/server/config/watcher.go | 4 +- pkg/mcs/scheduling/server/grpc_service.go | 2 +- pkg/mcs/scheduling/server/meta/watcher.go | 4 +- pkg/mcs/scheduling/server/rule/watcher.go | 4 +- pkg/mcs/scheduling/server/server.go | 8 +- pkg/mcs/utils/constant/constant.go | 4 +- pkg/member/election_leader.go | 2 +- pkg/schedule/coordinator.go | 2 +- .../schedulers/scheduler_controller.go | 2 +- pkg/tso/keyspace_group_manager.go | 2 +- pkg/utils/apiutil/serverapi/middleware.go | 4 +- server/api/admin.go | 2 +- server/api/server.go | 4 +- server/apiv2/handlers/micro_service.go | 4 +- server/cluster/cluster.go | 22 +- server/config/config.go | 2 +- server/grpc_service.go | 4 +- server/server.go | 22 +- server/server_test.go | 4 +- tests/cluster.go | 44 ++-- tests/integrations/client/client_test.go | 2 +- .../mcs/discovery/register_test.go | 8 +- .../mcs/keyspace/tso_keyspace_group_test.go | 2 +- tests/integrations/mcs/members/member_test.go | 2 +- tests/integrations/mcs/scheduling/api_test.go | 46 ++-- .../mcs/scheduling/config_test.go | 14 +- .../integrations/mcs/scheduling/meta_test.go | 2 +- .../integrations/mcs/scheduling/rule_test.go | 4 +- .../mcs/scheduling/server_test.go | 38 ++-- tests/integrations/mcs/tso/api_test.go | 8 +- .../mcs/tso/keyspace_group_manager_test.go | 14 +- tests/integrations/mcs/tso/proxy_test.go | 2 +- tests/integrations/mcs/tso/server_test.go | 52 ++--- tests/integrations/tso/client_test.go | 6 +- tests/integrations/tso/consistency_test.go | 2 +- tests/integrations/tso/server_test.go | 2 +- tests/server/api/scheduler_test.go | 2 +- .../apiv2/handlers/tso_keyspace_group_test.go | 2 +- tests/server/server_test.go | 2 +- tests/testutil.go | 28 +-- tools/pd-ctl/pdctl/command/config_command.go | 24 +-- tools/pd-ctl/tests/config/config_test.go | 54 ++--- .../tests/keyspace/keyspace_group_test.go | 14 +- tools/pd-ctl/tests/keyspace/keyspace_test.go | 4 +- tools/pd-simulator/simulator/drive.go | 2 +- 61 files changed, 455 insertions(+), 469 deletions(-) rename client/servicediscovery/{mock_pd_service_discovery.go => mock_service_discovery.go} (57%) rename client/servicediscovery/{pd_service_discovery.go => service_discovery.go} (81%) rename client/servicediscovery/{pd_service_discovery_test.go => service_discovery_test.go} (96%) diff --git a/client/client.go b/client/client.go index 7aa28cbc0cd..e3d3f4e5b14 100644 --- a/client/client.go +++ b/client/client.go @@ -360,8 +360,8 @@ func newClientWithKeyspaceName( c := &client{ callerComponent: adjustCallerComponent(callerComponent), inner: &innerClient{ - // Create a PD service discovery with null keyspace id, then query the real id with the keyspace name, - // finally update the keyspace id to the PD service discovery for the following interactions. + // Create a service discovery with null keyspace id, then query the real id with the keyspace name, + // finally update the keyspace id to the service discovery for the following interactions. keyspaceID: constants.NullKeyspaceID, updateTokenConnectionCh: make(chan struct{}, 1), ctx: clientCtx, @@ -384,7 +384,7 @@ func newClientWithKeyspaceName( } c.inner.keyspaceID = keyspaceMeta.GetId() // c.keyspaceID is the source of truth for keyspace id. - c.inner.pdSvcDiscovery.SetKeyspaceID(c.inner.keyspaceID) + c.inner.serviceDiscovery.SetKeyspaceID(c.inner.keyspaceID) return nil } @@ -412,17 +412,17 @@ func (c *client) ResetTSOClient() { // GetClusterID returns the ClusterID. func (c *client) GetClusterID(context.Context) uint64 { - return c.inner.pdSvcDiscovery.GetClusterID() + return c.inner.serviceDiscovery.GetClusterID() } // GetLeaderURL returns the leader URL. func (c *client) GetLeaderURL() string { - return c.inner.pdSvcDiscovery.GetServingURL() + return c.inner.serviceDiscovery.GetServingURL() } // GetServiceDiscovery returns the client-side service discovery object func (c *client) GetServiceDiscovery() sd.ServiceDiscovery { - return c.inner.pdSvcDiscovery + return c.inner.serviceDiscovery } // UpdateOption updates the client option. @@ -438,7 +438,7 @@ func (c *client) UpdateOption(option opt.DynamicOption, value any) error { } case opt.EnableTSOFollowerProxy: if c.inner.getServiceMode() != pdpb.ServiceMode_PD_SVC_MODE { - return errors.New("[pd] tso follower proxy is only supported in PD service mode") + return errors.New("[pd] tso follower proxy is only supported in PD mode") } enable, ok := value.(bool) if !ok { @@ -485,7 +485,7 @@ func (c *client) GetAllMembers(ctx context.Context) ([]*pdpb.Member, error) { // getClientAndContext returns the leader pd client and the original context. If leader is unhealthy, it returns // follower pd client and the context which holds forward information. func (c *client) getClientAndContext(ctx context.Context) (pdpb.PDClient, context.Context) { - serviceClient := c.inner.pdSvcDiscovery.GetServiceClient() + serviceClient := c.inner.serviceDiscovery.GetServiceClient() if serviceClient == nil || serviceClient.GetClientConn() == nil { return nil, ctx } @@ -526,7 +526,7 @@ func (c *client) GetLocalTS(ctx context.Context, _ string) (physical int64, logi // GetMinTS implements the TSOClient interface. func (c *client) GetMinTS(ctx context.Context) (physical int64, logical int64, err error) { - // Handle compatibility issue in case of PD/API server doesn't support GetMinTS API. + // Handle compatibility issue in case of PD/PD service doesn't support GetMinTS API. serviceMode := c.inner.getServiceMode() switch serviceMode { case pdpb.ServiceMode_UNKNOWN_SVC_MODE: @@ -598,7 +598,7 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs var resp *pdpb.GetRegionResponse for _, url := range memberURLs { - conn, err := c.inner.pdSvcDiscovery.GetOrCreateGRPCConn(url) + conn, err := c.inner.serviceDiscovery.GetOrCreateGRPCConn(url) if err != nil { log.Error("[pd] can't get grpc connection", zap.String("member-URL", url), errs.ZapError(err)) continue @@ -619,7 +619,7 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs if resp == nil { metrics.CmdFailedDurationGetRegion.Observe(time.Since(start).Seconds()) - c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.inner.serviceDiscovery.ScheduleCheckMemberChanged() errorMsg := fmt.Sprintf("[pd] can't get region info from member URLs: %+v", memberURLs) return nil, errors.WithStack(errors.New(errorMsg)) } @@ -1150,7 +1150,7 @@ func (c *client) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...o func (c *client) requestHeader() *pdpb.RequestHeader { return &pdpb.RequestHeader{ - ClusterId: c.inner.pdSvcDiscovery.GetClusterID(), + ClusterId: c.inner.serviceDiscovery.GetClusterID(), CallerId: string(caller.GetCallerID()), CallerComponent: string(c.callerComponent), } @@ -1334,7 +1334,7 @@ func (c *client) respForErr(observer prometheus.Observer, start time.Time, err e if err != nil || header.GetError() != nil { observer.Observe(time.Since(start).Seconds()) if err != nil { - c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.inner.serviceDiscovery.ScheduleCheckMemberChanged() return errors.WithStack(err) } return errors.WithStack(errors.New(header.GetError().String())) diff --git a/client/clients/tso/client.go b/client/clients/tso/client.go index d24dba52394..7bc768ee21b 100644 --- a/client/clients/tso/client.go +++ b/client/clients/tso/client.go @@ -563,7 +563,7 @@ func (c *Cli) DispatchRequest(request *Request) (bool, error) { // Client is closed, no need to retry. return false, request.clientCtx.Err() case <-c.ctx.Done(): - // tsoClient is closed due to the PD service mode switch, which is retryable. + // tsoClient is closed due to the service mode switch, which is retryable. return true, c.ctx.Err() default: // This failpoint will increase the possibility that the request is sent to a closed dispatcher. diff --git a/client/clients/tso/dispatcher_test.go b/client/clients/tso/dispatcher_test.go index 7e5554c7c7b..6d8f94791e6 100644 --- a/client/clients/tso/dispatcher_test.go +++ b/client/clients/tso/dispatcher_test.go @@ -53,7 +53,7 @@ func (m *mockTSOServiceProvider) getOption() *opt.Option { } func (*mockTSOServiceProvider) getServiceDiscovery() sd.ServiceDiscovery { - return sd.NewMockPDServiceDiscovery([]string{mockStreamURL}, nil) + return sd.NewMockServiceDiscovery([]string{mockStreamURL}, nil) } func (m *mockTSOServiceProvider) getConnectionCtxMgr() *cctx.Manager[*tsoStream] { diff --git a/client/http/client.go b/client/http/client.go index fa9801cf764..b7109166a30 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -245,7 +245,7 @@ func (ci *clientInner) doRequest( if readErr != nil { logFields = append(logFields, zap.NamedError("read-body-error", err)) } else { - // API server will return a JSON body containing the detailed error message + // PD service will return a JSON body containing the detailed error message // when the status code is not `http.StatusOK` 200. bs = bytes.TrimSpace(bs) logFields = append(logFields, zap.ByteString("body", bs)) @@ -304,7 +304,7 @@ func WithMetrics( } } -// NewClientWithServiceDiscovery creates a PD HTTP client with the given PD service discovery. +// NewClientWithServiceDiscovery creates a PD HTTP client with the given service discovery. func NewClientWithServiceDiscovery( source string, sd sd.ServiceDiscovery, @@ -332,7 +332,7 @@ func NewClient( for _, opt := range opts { opt(c) } - sd := sd.NewDefaultPDServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) + sd := sd.NewDefaultServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) if err := sd.Init(); err != nil { log.Error("[pd] init service discovery failed", zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) @@ -420,7 +420,7 @@ func NewHTTPClientWithRequestChecker(checker requestChecker) *http.Client { } } -// newClientWithMockServiceDiscovery creates a new PD HTTP client with a mock PD service discovery. +// newClientWithMockServiceDiscovery creates a new PD HTTP client with a mock service discovery. func newClientWithMockServiceDiscovery( source string, pdAddrs []string, @@ -432,7 +432,7 @@ func newClientWithMockServiceDiscovery( for _, opt := range opts { opt(c) } - sd := sd.NewMockPDServiceDiscovery(pdAddrs, c.inner.tlsConf) + sd := sd.NewMockServiceDiscovery(pdAddrs, c.inner.tlsConf) if err := sd.Init(); err != nil { log.Error("[pd] init mock service discovery failed", zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) diff --git a/client/inner_client.go b/client/inner_client.go index 404cbcf0b80..8379b6a51a9 100644 --- a/client/inner_client.go +++ b/client/inner_client.go @@ -26,10 +26,10 @@ const ( ) type innerClient struct { - keyspaceID uint32 - svrUrls []string - pdSvcDiscovery sd.ServiceDiscovery - tokenDispatcher *tokenDispatcher + keyspaceID uint32 + svrUrls []string + serviceDiscovery sd.ServiceDiscovery + tokenDispatcher *tokenDispatcher // For service mode switching. serviceModeKeeper @@ -45,13 +45,13 @@ type innerClient struct { } func (c *innerClient) init(updateKeyspaceIDCb sd.UpdateKeyspaceIDFunc) error { - c.pdSvcDiscovery = sd.NewPDServiceDiscovery( + c.serviceDiscovery = sd.NewServiceDiscovery( c.ctx, c.cancel, &c.wg, c.setServiceMode, updateKeyspaceIDCb, c.keyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { c.cancel() - if c.pdSvcDiscovery != nil { - c.pdSvcDiscovery.Close() + if c.serviceDiscovery != nil { + c.serviceDiscovery.Close() } return err } @@ -92,10 +92,10 @@ func (c *innerClient) resetTSOClientLocked(mode pdpb.ServiceMode) { switch mode { case pdpb.ServiceMode_PD_SVC_MODE: newTSOCli = tso.NewClient(c.ctx, c.option, - c.pdSvcDiscovery, &tso.PDStreamBuilderFactory{}) + c.serviceDiscovery, &tso.PDStreamBuilderFactory{}) case pdpb.ServiceMode_API_SVC_MODE: newTSOSvcDiscovery = sd.NewTSOServiceDiscovery( - c.ctx, c, c.pdSvcDiscovery, + c.ctx, c, c.serviceDiscovery, c.keyspaceID, c.tlsCfg, c.option) // At this point, the keyspace group isn't known yet. Starts from the default keyspace group, // and will be updated later. @@ -119,12 +119,12 @@ func (c *innerClient) resetTSOClientLocked(mode pdpb.ServiceMode) { oldTSOClient.Close() // Replace the old TSO service discovery if needed. oldTSOSvcDiscovery := c.tsoSvcDiscovery - // If newTSOSvcDiscovery is nil, that's expected, as it means we are switching to PD service mode and + // If newTSOSvcDiscovery is nil, that's expected, as it means we are switching to PD mode and // no tso microservice discovery is needed. c.tsoSvcDiscovery = newTSOSvcDiscovery // Close the old TSO service discovery safely after both the old client and service discovery are replaced. if oldTSOSvcDiscovery != nil { - // We are switching from API service mode to PD service mode, so delete the old tso microservice discovery. + // We are switching from PD service mode to PD mode, so delete the old tso microservice discovery. oldTSOSvcDiscovery.Close() } } @@ -153,7 +153,7 @@ func (c *innerClient) close() { c.wg.Wait() c.serviceModeKeeper.close() - c.pdSvcDiscovery.Close() + c.serviceDiscovery.Close() if c.tokenDispatcher != nil { tokenErr := errors.WithStack(errs.ErrClosing) @@ -169,12 +169,12 @@ func (c *innerClient) setup() error { } // Init the client base. - if err := c.pdSvcDiscovery.Init(); err != nil { + if err := c.serviceDiscovery.Init(); err != nil { return err } // Register callbacks - c.pdSvcDiscovery.AddServingURLSwitchedCallback(c.scheduleUpdateTokenConnection) + c.serviceDiscovery.AddServingURLSwitchedCallback(c.scheduleUpdateTokenConnection) // Create dispatchers c.createTokenDispatcher() @@ -186,12 +186,12 @@ func (c *innerClient) setup() error { func (c *innerClient) getRegionAPIClientAndContext(ctx context.Context, allowFollower bool) (sd.ServiceClient, context.Context) { var serviceClient sd.ServiceClient if allowFollower { - serviceClient = c.pdSvcDiscovery.GetServiceClientByKind(sd.UniversalAPIKind) + serviceClient = c.serviceDiscovery.GetServiceClientByKind(sd.UniversalAPIKind) if serviceClient != nil { return serviceClient, serviceClient.BuildGRPCTargetContext(ctx, !allowFollower) } } - serviceClient = c.pdSvcDiscovery.GetServiceClient() + serviceClient = c.serviceDiscovery.GetServiceClient() if serviceClient == nil || serviceClient.GetClientConn() == nil { return nil, ctx } @@ -201,12 +201,12 @@ func (c *innerClient) getRegionAPIClientAndContext(ctx context.Context, allowFol // gRPCErrorHandler is used to handle the gRPC error returned by the resource manager service. func (c *innerClient) gRPCErrorHandler(err error) { if errs.IsLeaderChange(err) { - c.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.serviceDiscovery.ScheduleCheckMemberChanged() } } func (c *innerClient) getOrCreateGRPCConn() (*grpc.ClientConn, error) { - cc, err := c.pdSvcDiscovery.GetOrCreateGRPCConn(c.pdSvcDiscovery.GetServingURL()) + cc, err := c.serviceDiscovery.GetOrCreateGRPCConn(c.serviceDiscovery.GetServingURL()) if err != nil { return nil, err } diff --git a/client/keyspace_client.go b/client/keyspace_client.go index 84bc29054eb..507279e906c 100644 --- a/client/keyspace_client.go +++ b/client/keyspace_client.go @@ -41,7 +41,7 @@ type KeyspaceClient interface { // keyspaceClient returns the KeyspaceClient from current PD leader. func (c *client) keyspaceClient() keyspacepb.KeyspaceClient { - if client := c.inner.pdSvcDiscovery.GetServingEndpointClientConn(); client != nil { + if client := c.inner.serviceDiscovery.GetServingEndpointClientConn(); client != nil { return keyspacepb.NewKeyspaceClient(client) } return nil @@ -70,7 +70,7 @@ func (c *client) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.Key if err != nil { metrics.CmdFailedDurationLoadKeyspace.Observe(time.Since(start).Seconds()) - c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.inner.serviceDiscovery.ScheduleCheckMemberChanged() return nil, err } @@ -115,7 +115,7 @@ func (c *client) UpdateKeyspaceState(ctx context.Context, id uint32, state keysp if err != nil { metrics.CmdFailedDurationUpdateKeyspaceState.Observe(time.Since(start).Seconds()) - c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.inner.serviceDiscovery.ScheduleCheckMemberChanged() return nil, err } @@ -159,7 +159,7 @@ func (c *client) GetAllKeyspaces(ctx context.Context, startID uint32, limit uint if err != nil { metrics.CmdDurationGetAllKeyspaces.Observe(time.Since(start).Seconds()) - c.inner.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.inner.serviceDiscovery.ScheduleCheckMemberChanged() return nil, err } diff --git a/client/meta_storage_client.go b/client/meta_storage_client.go index fbabd60debd..7652884720d 100644 --- a/client/meta_storage_client.go +++ b/client/meta_storage_client.go @@ -33,7 +33,7 @@ import ( // metaStorageClient gets the meta storage client from current PD leader. func (c *innerClient) metaStorageClient() meta_storagepb.MetaStorageClient { - if client := c.pdSvcDiscovery.GetServingEndpointClientConn(); client != nil { + if client := c.serviceDiscovery.GetServingEndpointClientConn(); client != nil { return meta_storagepb.NewMetaStorageClient(client) } return nil @@ -74,7 +74,7 @@ func (c *innerClient) Put(ctx context.Context, key, value []byte, opts ...opt.Me Lease: options.Lease, PrevKv: options.PrevKv, } - ctx = grpcutil.BuildForwardContext(ctx, c.pdSvcDiscovery.GetServingURL()) + ctx = grpcutil.BuildForwardContext(ctx, c.serviceDiscovery.GetServingURL()) cli := c.metaStorageClient() if cli == nil { cancel() @@ -113,7 +113,7 @@ func (c *innerClient) Get(ctx context.Context, key []byte, opts ...opt.MetaStora Limit: options.Limit, Revision: options.Revision, } - ctx = grpcutil.BuildForwardContext(ctx, c.pdSvcDiscovery.GetServingURL()) + ctx = grpcutil.BuildForwardContext(ctx, c.serviceDiscovery.GetServingURL()) cli := c.metaStorageClient() if cli == nil { cancel() @@ -179,7 +179,7 @@ func (c *innerClient) respForMetaStorageErr(observer prometheus.Observer, start if err != nil || header.GetError() != nil { observer.Observe(time.Since(start).Seconds()) if err != nil { - c.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.serviceDiscovery.ScheduleCheckMemberChanged() return errors.WithStack(err) } return errors.WithStack(errors.New(header.GetError().String())) diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index 0c481631b93..3e4cd1a3cc8 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -331,7 +331,7 @@ func (c *innerClient) handleResourceTokenDispatcher(dispatcherCtx context.Contex // If the stream is still nil, return an error. if stream == nil { firstRequest.done <- errors.Errorf("failed to get the stream connection") - c.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.serviceDiscovery.ScheduleCheckMemberChanged() connection.reset() continue } @@ -343,7 +343,7 @@ func (c *innerClient) handleResourceTokenDispatcher(dispatcherCtx context.Contex default: } if err = c.processTokenRequests(stream, firstRequest); err != nil { - c.pdSvcDiscovery.ScheduleCheckMemberChanged() + c.serviceDiscovery.ScheduleCheckMemberChanged() connection.reset() log.Info("[resource_manager] token request error", zap.Error(err)) } diff --git a/client/servicediscovery/mock_pd_service_discovery.go b/client/servicediscovery/mock_service_discovery.go similarity index 57% rename from client/servicediscovery/mock_pd_service_discovery.go rename to client/servicediscovery/mock_service_discovery.go index 87b74ae2136..6ca649f4575 100644 --- a/client/servicediscovery/mock_pd_service_discovery.go +++ b/client/servicediscovery/mock_service_discovery.go @@ -21,24 +21,24 @@ import ( "google.golang.org/grpc" ) -var _ ServiceDiscovery = (*mockPDServiceDiscovery)(nil) +var _ ServiceDiscovery = (*mockServiceDiscovery)(nil) -type mockPDServiceDiscovery struct { +type mockServiceDiscovery struct { urls []string tlsCfg *tls.Config clients []ServiceClient } -// NewMockPDServiceDiscovery creates a mock PD service discovery. -func NewMockPDServiceDiscovery(urls []string, tlsCfg *tls.Config) *mockPDServiceDiscovery { - return &mockPDServiceDiscovery{ +// NewMockServiceDiscovery creates a mock service discovery. +func NewMockServiceDiscovery(urls []string, tlsCfg *tls.Config) *mockServiceDiscovery { + return &mockServiceDiscovery{ urls: urls, tlsCfg: tlsCfg, } } // Init directly creates the service clients with the given URLs. -func (m *mockPDServiceDiscovery) Init() error { +func (m *mockServiceDiscovery) Init() error { m.clients = make([]ServiceClient, 0, len(m.urls)) for _, url := range m.urls { m.clients = append(m.clients, newPDServiceClient(url, m.urls[0], nil, false)) @@ -47,61 +47,61 @@ func (m *mockPDServiceDiscovery) Init() error { } // Close clears the service clients. -func (m *mockPDServiceDiscovery) Close() { +func (m *mockServiceDiscovery) Close() { clear(m.clients) } -// GetAllServiceClients returns all service clients init in the mock PD service discovery. -func (m *mockPDServiceDiscovery) GetAllServiceClients() []ServiceClient { +// GetAllServiceClients returns all service clients init in the mock service discovery. +func (m *mockServiceDiscovery) GetAllServiceClients() []ServiceClient { return m.clients } // GetClusterID implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetClusterID() uint64 { return 0 } +func (*mockServiceDiscovery) GetClusterID() uint64 { return 0 } // GetKeyspaceID implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetKeyspaceID() uint32 { return 0 } +func (*mockServiceDiscovery) GetKeyspaceID() uint32 { return 0 } // SetKeyspaceID implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) SetKeyspaceID(uint32) {} +func (*mockServiceDiscovery) SetKeyspaceID(uint32) {} // GetKeyspaceGroupID implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetKeyspaceGroupID() uint32 { return 0 } +func (*mockServiceDiscovery) GetKeyspaceGroupID() uint32 { return 0 } // GetServiceURLs implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetServiceURLs() []string { return nil } +func (*mockServiceDiscovery) GetServiceURLs() []string { return nil } // GetServingEndpointClientConn implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { return nil } +func (*mockServiceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { return nil } // GetClientConns implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetClientConns() *sync.Map { return nil } +func (*mockServiceDiscovery) GetClientConns() *sync.Map { return nil } // GetServingURL implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetServingURL() string { return "" } +func (*mockServiceDiscovery) GetServingURL() string { return "" } // GetBackupURLs implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetBackupURLs() []string { return nil } +func (*mockServiceDiscovery) GetBackupURLs() []string { return nil } // GetServiceClient implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetServiceClient() ServiceClient { return nil } +func (*mockServiceDiscovery) GetServiceClient() ServiceClient { return nil } // GetServiceClientByKind implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetServiceClientByKind(APIKind) ServiceClient { return nil } +func (*mockServiceDiscovery) GetServiceClientByKind(APIKind) ServiceClient { return nil } // GetOrCreateGRPCConn implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) GetOrCreateGRPCConn(string) (*grpc.ClientConn, error) { +func (*mockServiceDiscovery) GetOrCreateGRPCConn(string) (*grpc.ClientConn, error) { return nil, nil } // ScheduleCheckMemberChanged implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) ScheduleCheckMemberChanged() {} +func (*mockServiceDiscovery) ScheduleCheckMemberChanged() {} // CheckMemberChanged implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) CheckMemberChanged() error { return nil } +func (*mockServiceDiscovery) CheckMemberChanged() error { return nil } // AddServingURLSwitchedCallback implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) AddServingURLSwitchedCallback(...func()) {} +func (*mockServiceDiscovery) AddServingURLSwitchedCallback(...func()) {} // AddServiceURLsSwitchedCallback implements the ServiceDiscovery interface. -func (*mockPDServiceDiscovery) AddServiceURLsSwitchedCallback(...func()) {} +func (*mockServiceDiscovery) AddServiceURLsSwitchedCallback(...func()) {} diff --git a/client/servicediscovery/pd_service_discovery.go b/client/servicediscovery/service_discovery.go similarity index 81% rename from client/servicediscovery/pd_service_discovery.go rename to client/servicediscovery/service_discovery.go index 931f950c6d1..bef80e28a37 100644 --- a/client/servicediscovery/pd_service_discovery.go +++ b/client/servicediscovery/service_discovery.go @@ -157,11 +157,11 @@ type ServiceClient interface { } var ( - _ ServiceClient = (*pdServiceClient)(nil) - _ ServiceClient = (*pdServiceAPIClient)(nil) + _ ServiceClient = (*serviceClient)(nil) + _ ServiceClient = (*serviceAPIClient)(nil) ) -type pdServiceClient struct { +type serviceClient struct { url string conn *grpc.ClientConn isLeader bool @@ -171,10 +171,10 @@ type pdServiceClient struct { } // NOTE: In the current implementation, the URL passed in is bound to have a scheme, -// because it is processed in `newPDServiceDiscovery`, and the url returned by etcd member owns the scheme. +// because it is processed in `newServiceDiscovery`, and the url returned by etcd member owns the scheme. // When testing, the URL is also bound to have a scheme. func newPDServiceClient(url, leaderURL string, conn *grpc.ClientConn, isLeader bool) ServiceClient { - cli := &pdServiceClient{ + cli := &serviceClient{ url: url, conn: conn, isLeader: isLeader, @@ -187,7 +187,7 @@ func newPDServiceClient(url, leaderURL string, conn *grpc.ClientConn, isLeader b } // GetURL implements ServiceClient. -func (c *pdServiceClient) GetURL() string { +func (c *serviceClient) GetURL() string { if c == nil { return "" } @@ -195,7 +195,7 @@ func (c *pdServiceClient) GetURL() string { } // BuildGRPCTargetContext implements ServiceClient. -func (c *pdServiceClient) BuildGRPCTargetContext(ctx context.Context, toLeader bool) context.Context { +func (c *serviceClient) BuildGRPCTargetContext(ctx context.Context, toLeader bool) context.Context { if c == nil || c.isLeader { return ctx } @@ -206,7 +206,7 @@ func (c *pdServiceClient) BuildGRPCTargetContext(ctx context.Context, toLeader b } // IsConnectedToLeader implements ServiceClient. -func (c *pdServiceClient) IsConnectedToLeader() bool { +func (c *serviceClient) IsConnectedToLeader() bool { if c == nil { return false } @@ -214,14 +214,14 @@ func (c *pdServiceClient) IsConnectedToLeader() bool { } // Available implements ServiceClient. -func (c *pdServiceClient) Available() bool { +func (c *serviceClient) Available() bool { if c == nil { return false } return !c.networkFailure.Load() } -func (c *pdServiceClient) checkNetworkAvailable(ctx context.Context) { +func (c *serviceClient) checkNetworkAvailable(ctx context.Context) { if c == nil || c.conn == nil { return } @@ -242,7 +242,7 @@ func (c *pdServiceClient) checkNetworkAvailable(ctx context.Context) { } // GetClientConn implements ServiceClient. -func (c *pdServiceClient) GetClientConn() *grpc.ClientConn { +func (c *serviceClient) GetClientConn() *grpc.ClientConn { if c == nil { return nil } @@ -250,7 +250,7 @@ func (c *pdServiceClient) GetClientConn() *grpc.ClientConn { } // NeedRetry implements ServiceClient. -func (c *pdServiceClient) NeedRetry(pdErr *pdpb.Error, err error) bool { +func (c *serviceClient) NeedRetry(pdErr *pdpb.Error, err error) bool { if c.IsConnectedToLeader() { return false } @@ -267,9 +267,9 @@ func regionAPIErrorFn(pdErr *pdpb.Error) bool { return pdErr.GetType() == pdpb.ErrorType_REGION_NOT_FOUND } -// pdServiceAPIClient is a specific API client for PD service. -// It extends the pdServiceClient and adds additional fields for managing availability -type pdServiceAPIClient struct { +// serviceAPIClient is a specific API client for service. +// It extends the serviceClient and adds additional fields for managing availability +type serviceAPIClient struct { ServiceClient fn errFn @@ -278,19 +278,19 @@ type pdServiceAPIClient struct { } func newPDServiceAPIClient(client ServiceClient, f errFn) ServiceClient { - return &pdServiceAPIClient{ + return &serviceAPIClient{ ServiceClient: client, fn: f, } } // Available implements ServiceClient. -func (c *pdServiceAPIClient) Available() bool { +func (c *serviceAPIClient) Available() bool { return c.ServiceClient.Available() && !c.unavailable.Load() } // markAsAvailable is used to try to mark the client as available if unavailable status is expired. -func (c *pdServiceAPIClient) markAsAvailable() { +func (c *serviceAPIClient) markAsAvailable() { if !c.unavailable.Load() { return } @@ -301,7 +301,7 @@ func (c *pdServiceAPIClient) markAsAvailable() { } // NeedRetry implements ServiceClient. -func (c *pdServiceAPIClient) NeedRetry(pdErr *pdpb.Error, err error) bool { +func (c *serviceAPIClient) NeedRetry(pdErr *pdpb.Error, err error) bool { if c.IsConnectedToLeader() { return false } @@ -317,43 +317,43 @@ func (c *pdServiceAPIClient) NeedRetry(pdErr *pdpb.Error, err error) bool { return true } -// pdServiceBalancerNode is a balancer node for PD service. -// It extends the pdServiceClient and adds additional fields for the next polling client in the chain. -type pdServiceBalancerNode struct { - *pdServiceAPIClient - next *pdServiceBalancerNode +// serviceBalancerNode is a balancer node for PD. +// It extends the serviceClient and adds additional fields for the next polling client in the chain. +type serviceBalancerNode struct { + *serviceAPIClient + next *serviceBalancerNode } -// pdServiceBalancer is a load balancer for PD service clients. -// It is used to balance the request to all servers and manage the connections to multiple PD service nodes. -type pdServiceBalancer struct { +// serviceBalancer is a load balancer for clients. +// It is used to balance the request to all servers and manage the connections to multiple nodes. +type serviceBalancer struct { mu sync.Mutex - now *pdServiceBalancerNode + now *serviceBalancerNode totalNode int errFn errFn } -func newPDServiceBalancer(fn errFn) *pdServiceBalancer { - return &pdServiceBalancer{ +func newServiceBalancer(fn errFn) *serviceBalancer { + return &serviceBalancer{ errFn: fn, } } -func (c *pdServiceBalancer) set(clients []ServiceClient) { +func (c *serviceBalancer) set(clients []ServiceClient) { c.mu.Lock() defer c.mu.Unlock() if len(clients) == 0 { return } c.totalNode = len(clients) - head := &pdServiceBalancerNode{ - pdServiceAPIClient: newPDServiceAPIClient(clients[c.totalNode-1], c.errFn).(*pdServiceAPIClient), + head := &serviceBalancerNode{ + serviceAPIClient: newPDServiceAPIClient(clients[c.totalNode-1], c.errFn).(*serviceAPIClient), } head.next = head last := head for i := c.totalNode - 2; i >= 0; i-- { - next := &pdServiceBalancerNode{ - pdServiceAPIClient: newPDServiceAPIClient(clients[i], c.errFn).(*pdServiceAPIClient), - next: head, + next := &serviceBalancerNode{ + serviceAPIClient: newPDServiceAPIClient(clients[i], c.errFn).(*serviceAPIClient), + next: head, } head = next last.next = head @@ -361,7 +361,7 @@ func (c *pdServiceBalancer) set(clients []ServiceClient) { c.now = head } -func (c *pdServiceBalancer) check() { +func (c *serviceBalancer) check() { c.mu.Lock() defer c.mu.Unlock() for range c.totalNode { @@ -370,11 +370,11 @@ func (c *pdServiceBalancer) check() { } } -func (c *pdServiceBalancer) next() { +func (c *serviceBalancer) next() { c.now = c.now.next } -func (c *pdServiceBalancer) get() (ret ServiceClient) { +func (c *serviceBalancer) get() (ret ServiceClient) { c.mu.Lock() defer c.mu.Unlock() i := 0 @@ -403,22 +403,22 @@ type TSOEventSource interface { } var ( - _ ServiceDiscovery = (*pdServiceDiscovery)(nil) - _ TSOEventSource = (*pdServiceDiscovery)(nil) + _ ServiceDiscovery = (*serviceDiscovery)(nil) + _ TSOEventSource = (*serviceDiscovery)(nil) ) -// pdServiceDiscovery is the service discovery client of PD/API service which is quorum based -type pdServiceDiscovery struct { +// serviceDiscovery is the service discovery client of PD/PD service which is quorum based +type serviceDiscovery struct { isInitialized bool urls atomic.Value // Store as []string // PD leader - leader atomic.Value // Store as pdServiceClient + leader atomic.Value // Store as serviceClient // PD follower - followers sync.Map // Store as map[string]pdServiceClient + followers sync.Map // Store as map[string]serviceClient // PD leader and PD followers - all atomic.Value // Store as []pdServiceClient - apiCandidateNodes [apiKindCount]*pdServiceBalancer + all atomic.Value // Store as []serviceClient + apiCandidateNodes [apiKindCount]*serviceBalancer // PD follower URLs. Only for tso. followerURLs atomic.Value // Store as []string @@ -450,17 +450,17 @@ type pdServiceDiscovery struct { option *opt.Option } -// NewDefaultPDServiceDiscovery returns a new default PD service discovery-based client. -func NewDefaultPDServiceDiscovery( +// NewDefaultServiceDiscovery returns a new default service discovery-based client. +func NewDefaultServiceDiscovery( ctx context.Context, cancel context.CancelFunc, urls []string, tlsCfg *tls.Config, ) ServiceDiscovery { var wg sync.WaitGroup - return NewPDServiceDiscovery(ctx, cancel, &wg, nil, nil, constants.DefaultKeyspaceID, urls, tlsCfg, opt.NewOption()) + return NewServiceDiscovery(ctx, cancel, &wg, nil, nil, constants.DefaultKeyspaceID, urls, tlsCfg, opt.NewOption()) } -// NewPDServiceDiscovery returns a new PD service discovery-based client. -func NewPDServiceDiscovery( +// NewServiceDiscovery returns a new service discovery-based client. +func NewServiceDiscovery( ctx context.Context, cancel context.CancelFunc, wg *sync.WaitGroup, serviceModeUpdateCb func(pdpb.ServiceMode), @@ -468,12 +468,12 @@ func NewPDServiceDiscovery( keyspaceID uint32, urls []string, tlsCfg *tls.Config, option *opt.Option, ) ServiceDiscovery { - pdsd := &pdServiceDiscovery{ + pdsd := &serviceDiscovery{ checkMembershipCh: make(chan struct{}, 1), ctx: ctx, cancel: cancel, wg: wg, - apiCandidateNodes: [apiKindCount]*pdServiceBalancer{newPDServiceBalancer(emptyErrorFn), newPDServiceBalancer(regionAPIErrorFn)}, + apiCandidateNodes: [apiKindCount]*serviceBalancer{newServiceBalancer(emptyErrorFn), newServiceBalancer(regionAPIErrorFn)}, serviceModeUpdateCb: serviceModeUpdateCb, updateKeyspaceIDFunc: updateKeyspaceIDFunc, keyspaceID: keyspaceID, @@ -485,8 +485,8 @@ func NewPDServiceDiscovery( return pdsd } -// Init initializes the PD service discovery. -func (c *pdServiceDiscovery) Init() error { +// Init initializes the service discovery. +func (c *serviceDiscovery) Init() error { if c.isInitialized { return nil } @@ -523,7 +523,7 @@ func (c *pdServiceDiscovery) Init() error { return nil } -func (c *pdServiceDiscovery) initRetry(f func() error) error { +func (c *serviceDiscovery) initRetry(f func() error) error { var err error ticker := time.NewTicker(time.Second) defer ticker.Stop() @@ -540,7 +540,7 @@ func (c *pdServiceDiscovery) initRetry(f func() error) error { return errors.WithStack(err) } -func (c *pdServiceDiscovery) updateMemberLoop() { +func (c *serviceDiscovery) updateMemberLoop() { defer c.wg.Done() ctx, cancel := context.WithCancel(c.ctx) @@ -564,7 +564,7 @@ func (c *pdServiceDiscovery) updateMemberLoop() { } } -func (c *pdServiceDiscovery) updateServiceModeLoop() { +func (c *serviceDiscovery) updateServiceModeLoop() { defer c.wg.Done() failpoint.Inject("skipUpdateServiceMode", func() { failpoint.Return() @@ -596,7 +596,7 @@ func (c *pdServiceDiscovery) updateServiceModeLoop() { } } -func (c *pdServiceDiscovery) memberHealthCheckLoop() { +func (c *serviceDiscovery) memberHealthCheckLoop() { defer c.wg.Done() memberCheckLoopCtx, memberCheckLoopCancel := context.WithCancel(c.ctx) @@ -616,19 +616,19 @@ func (c *pdServiceDiscovery) memberHealthCheckLoop() { } } -func (c *pdServiceDiscovery) checkLeaderHealth(ctx context.Context) { +func (c *serviceDiscovery) checkLeaderHealth(ctx context.Context) { ctx, cancel := context.WithTimeout(ctx, c.option.Timeout) defer cancel() leader := c.getLeaderServiceClient() leader.checkNetworkAvailable(ctx) } -func (c *pdServiceDiscovery) checkFollowerHealth(ctx context.Context) { +func (c *serviceDiscovery) checkFollowerHealth(ctx context.Context) { c.followers.Range(func(_, value any) bool { // To ensure that the leader's healthy check is not delayed, shorten the duration. ctx, cancel := context.WithTimeout(ctx, MemberHealthCheckInterval/3) defer cancel() - serviceClient := value.(*pdServiceClient) + serviceClient := value.(*serviceClient) serviceClient.checkNetworkAvailable(ctx) return true }) @@ -638,12 +638,12 @@ func (c *pdServiceDiscovery) checkFollowerHealth(ctx context.Context) { } // Close releases all resources. -func (c *pdServiceDiscovery) Close() { +func (c *serviceDiscovery) Close() { if c == nil { return } c.closeOnce.Do(func() { - log.Info("[pd] close pd service discovery client") + log.Info("[pd] close service discovery client") c.clientConns.Range(func(key, cc any) bool { if err := cc.(*grpc.ClientConn).Close(); err != nil { log.Error("[pd] failed to close grpc clientConn", errs.ZapError(errs.ErrCloseGRPCConn, err)) @@ -655,28 +655,28 @@ func (c *pdServiceDiscovery) Close() { } // GetClusterID returns the ClusterID. -func (c *pdServiceDiscovery) GetClusterID() uint64 { +func (c *serviceDiscovery) GetClusterID() uint64 { return c.clusterID } // GetKeyspaceID returns the ID of the keyspace -func (c *pdServiceDiscovery) GetKeyspaceID() uint32 { +func (c *serviceDiscovery) GetKeyspaceID() uint32 { return c.keyspaceID } // SetKeyspaceID sets the ID of the keyspace -func (c *pdServiceDiscovery) SetKeyspaceID(keyspaceID uint32) { +func (c *serviceDiscovery) SetKeyspaceID(keyspaceID uint32) { c.keyspaceID = keyspaceID } // GetKeyspaceGroupID returns the ID of the keyspace group -func (*pdServiceDiscovery) GetKeyspaceGroupID() uint32 { - // PD/API service only supports the default keyspace group +func (*serviceDiscovery) GetKeyspaceGroupID() uint32 { + // PD only supports the default keyspace group return constants.DefaultKeyspaceGroupID } // DiscoverMicroservice discovers the microservice with the specified type and returns the server urls. -func (c *pdServiceDiscovery) discoverMicroservice(svcType serviceType) (urls []string, err error) { +func (c *serviceDiscovery) discoverMicroservice(svcType serviceType) (urls []string, err error) { switch svcType { case apiService: urls = c.GetServiceURLs() @@ -703,14 +703,14 @@ func (c *pdServiceDiscovery) discoverMicroservice(svcType serviceType) (urls []s // GetServiceURLs returns the URLs of the servers. // For testing use. It should only be called when the client is closed. -func (c *pdServiceDiscovery) GetServiceURLs() []string { +func (c *serviceDiscovery) GetServiceURLs() []string { return c.urls.Load().([]string) } // GetServingEndpointClientConn returns the grpc client connection of the serving endpoint // which is the leader in a quorum-based cluster or the primary in a primary/secondary // configured cluster. -func (c *pdServiceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { +func (c *serviceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { if cc, ok := c.clientConns.Load(c.getLeaderURL()); ok { return cc.(*grpc.ClientConn) } @@ -718,32 +718,32 @@ func (c *pdServiceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { } // GetClientConns returns the mapping {URL -> a gRPC connection} -func (c *pdServiceDiscovery) GetClientConns() *sync.Map { +func (c *serviceDiscovery) GetClientConns() *sync.Map { return &c.clientConns } // GetServingURL returns the leader url -func (c *pdServiceDiscovery) GetServingURL() string { +func (c *serviceDiscovery) GetServingURL() string { return c.getLeaderURL() } // GetBackupURLs gets the URLs of the current reachable followers // in a quorum-based cluster. Used for tso currently. -func (c *pdServiceDiscovery) GetBackupURLs() []string { +func (c *serviceDiscovery) GetBackupURLs() []string { return c.getFollowerURLs() } // getLeaderServiceClient returns the leader ServiceClient. -func (c *pdServiceDiscovery) getLeaderServiceClient() *pdServiceClient { +func (c *serviceDiscovery) getLeaderServiceClient() *serviceClient { leader := c.leader.Load() if leader == nil { return nil } - return leader.(*pdServiceClient) + return leader.(*serviceClient) } // GetServiceClientByKind returns ServiceClient of the specific kind. -func (c *pdServiceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient { +func (c *serviceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient { client := c.apiCandidateNodes[kind].get() if client == nil { return nil @@ -752,7 +752,7 @@ func (c *pdServiceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient } // GetServiceClient returns the leader/primary ServiceClient if it is healthy. -func (c *pdServiceDiscovery) GetServiceClient() ServiceClient { +func (c *serviceDiscovery) GetServiceClient() ServiceClient { leaderClient := c.getLeaderServiceClient() if c.option.EnableForwarding && !leaderClient.Available() { if followerClient := c.GetServiceClientByKind(ForwardAPIKind); followerClient != nil { @@ -767,7 +767,7 @@ func (c *pdServiceDiscovery) GetServiceClient() ServiceClient { } // GetAllServiceClients implements ServiceDiscovery -func (c *pdServiceDiscovery) GetAllServiceClients() []ServiceClient { +func (c *serviceDiscovery) GetAllServiceClients() []ServiceClient { all := c.all.Load() if all == nil { return nil @@ -778,7 +778,7 @@ func (c *pdServiceDiscovery) GetAllServiceClients() []ServiceClient { // ScheduleCheckMemberChanged is used to check if there is any membership // change among the leader and the followers. -func (c *pdServiceDiscovery) ScheduleCheckMemberChanged() { +func (c *serviceDiscovery) ScheduleCheckMemberChanged() { select { case c.checkMembershipCh <- struct{}{}: default: @@ -787,24 +787,24 @@ func (c *pdServiceDiscovery) ScheduleCheckMemberChanged() { // CheckMemberChanged Immediately check if there is any membership change among the leader/followers in a // quorum-based cluster or among the primary/secondaries in a primary/secondary configured cluster. -func (c *pdServiceDiscovery) CheckMemberChanged() error { +func (c *serviceDiscovery) CheckMemberChanged() error { return c.updateMember() } // AddServingURLSwitchedCallback adds callbacks which will be called // when the leader is switched. -func (c *pdServiceDiscovery) AddServingURLSwitchedCallback(callbacks ...func()) { +func (c *serviceDiscovery) AddServingURLSwitchedCallback(callbacks ...func()) { c.leaderSwitchedCbs = append(c.leaderSwitchedCbs, callbacks...) } // AddServiceURLsSwitchedCallback adds callbacks which will be called when // any leader/follower is changed. -func (c *pdServiceDiscovery) AddServiceURLsSwitchedCallback(callbacks ...func()) { +func (c *serviceDiscovery) AddServiceURLsSwitchedCallback(callbacks ...func()) { c.membersChangedCbs = append(c.membersChangedCbs, callbacks...) } // SetTSOLeaderURLUpdatedCallback adds a callback which will be called when the TSO leader is updated. -func (c *pdServiceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderURLUpdatedFunc) { +func (c *serviceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderURLUpdatedFunc) { url := c.getLeaderURL() if len(url) > 0 { if err := callback(url); err != nil { @@ -815,12 +815,12 @@ func (c *pdServiceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderUR } // getLeaderURL returns the leader URL. -func (c *pdServiceDiscovery) getLeaderURL() string { +func (c *serviceDiscovery) getLeaderURL() string { return c.getLeaderServiceClient().GetURL() } // getFollowerURLs returns the follower URLs. -func (c *pdServiceDiscovery) getFollowerURLs() []string { +func (c *serviceDiscovery) getFollowerURLs() []string { followerURLs := c.followerURLs.Load() if followerURLs == nil { return []string{} @@ -828,7 +828,7 @@ func (c *pdServiceDiscovery) getFollowerURLs() []string { return followerURLs.([]string) } -func (c *pdServiceDiscovery) initClusterID() error { +func (c *serviceDiscovery) initClusterID() error { ctx, cancel := context.WithCancel(c.ctx) defer cancel() clusterID := uint64(0) @@ -855,7 +855,7 @@ func (c *pdServiceDiscovery) initClusterID() error { return nil } -func (c *pdServiceDiscovery) checkServiceModeChanged() error { +func (c *serviceDiscovery) checkServiceModeChanged() error { leaderURL := c.getLeaderURL() if len(leaderURL) == 0 { return errors.New("no leader found") @@ -883,7 +883,7 @@ func (c *pdServiceDiscovery) checkServiceModeChanged() error { return nil } -func (c *pdServiceDiscovery) updateMember() error { +func (c *serviceDiscovery) updateMember() error { for _, url := range c.GetServiceURLs() { members, err := c.getMembers(c.ctx, url, UpdateMemberTimeout) // Check the cluster ID. @@ -916,7 +916,7 @@ func (c *pdServiceDiscovery) updateMember() error { return errs.ErrClientGetMember.FastGenByArgs() } -func (c *pdServiceDiscovery) getClusterInfo(ctx context.Context, url string, timeout time.Duration) (*pdpb.GetClusterInfoResponse, error) { +func (c *serviceDiscovery) getClusterInfo(ctx context.Context, url string, timeout time.Duration) (*pdpb.GetClusterInfoResponse, error) { ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() cc, err := c.GetOrCreateGRPCConn(url) @@ -935,7 +935,7 @@ func (c *pdServiceDiscovery) getClusterInfo(ctx context.Context, url string, tim return clusterInfo, nil } -func (c *pdServiceDiscovery) getMembers(ctx context.Context, url string, timeout time.Duration) (*pdpb.GetMembersResponse, error) { +func (c *serviceDiscovery) getMembers(ctx context.Context, url string, timeout time.Duration) (*pdpb.GetMembersResponse, error) { ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() cc, err := c.GetOrCreateGRPCConn(url) @@ -954,7 +954,7 @@ func (c *pdServiceDiscovery) getMembers(ctx context.Context, url string, timeout return members, nil } -func (c *pdServiceDiscovery) updateURLs(members []*pdpb.Member) { +func (c *serviceDiscovery) updateURLs(members []*pdpb.Member) { urls := make([]string, 0, len(members)) for _, m := range members { urls = append(urls, m.GetClientUrls()...) @@ -974,7 +974,7 @@ func (c *pdServiceDiscovery) updateURLs(members []*pdpb.Member) { log.Info("[pd] update member urls", zap.Strings("old-urls", oldURLs), zap.Strings("new-urls", urls)) } -func (c *pdServiceDiscovery) switchLeader(url string) (bool, error) { +func (c *serviceDiscovery) switchLeader(url string) (bool, error) { oldLeader := c.getLeaderServiceClient() if url == oldLeader.GetURL() && oldLeader.GetClientConn() != nil { return false, nil @@ -999,10 +999,10 @@ func (c *pdServiceDiscovery) switchLeader(url string) (bool, error) { return true, err } -func (c *pdServiceDiscovery) updateFollowers(members []*pdpb.Member, leaderID uint64, leaderURL string) (changed bool) { - followers := make(map[string]*pdServiceClient) +func (c *serviceDiscovery) updateFollowers(members []*pdpb.Member, leaderID uint64, leaderURL string) (changed bool) { + followers := make(map[string]*serviceClient) c.followers.Range(func(key, value any) bool { - followers[key.(string)] = value.(*pdServiceClient) + followers[key.(string)] = value.(*serviceClient) return true }) var followerURLs []string @@ -1015,7 +1015,7 @@ func (c *pdServiceDiscovery) updateFollowers(members []*pdpb.Member, leaderID ui // FIXME: How to safely compare urls(also for leader)? For now, only allows one client url. url := tlsutil.PickMatchedURL(member.GetClientUrls(), c.tlsCfg) if client, ok := c.followers.Load(url); ok { - if client.(*pdServiceClient).GetClientConn() == nil { + if client.(*serviceClient).GetClientConn() == nil { conn, err := c.GetOrCreateGRPCConn(url) if err != nil || conn == nil { log.Warn("[pd] failed to connect follower", zap.String("follower", url), errs.ZapError(err)) @@ -1048,7 +1048,7 @@ func (c *pdServiceDiscovery) updateFollowers(members []*pdpb.Member, leaderID ui return } -func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader *pdpb.Member) error { +func (c *serviceDiscovery) updateServiceClient(members []*pdpb.Member, leader *pdpb.Member) error { // FIXME: How to safely compare leader urls? For now, only allows one client url. leaderURL := tlsutil.PickMatchedURL(leader.GetClientUrls(), c.tlsCfg) leaderChanged, err := c.switchLeader(leaderURL) @@ -1064,7 +1064,7 @@ func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader clients = append(clients, leaderClient) } c.followers.Range(func(_, value any) bool { - clients = append(clients, value.(*pdServiceClient)) + clients = append(clients, value.(*serviceClient)) return true }) c.all.Store(clients) @@ -1076,6 +1076,6 @@ func (c *pdServiceDiscovery) updateServiceClient(members []*pdpb.Member, leader } // GetOrCreateGRPCConn returns the corresponding grpc client connection of the given URL. -func (c *pdServiceDiscovery) GetOrCreateGRPCConn(url string) (*grpc.ClientConn, error) { +func (c *serviceDiscovery) GetOrCreateGRPCConn(url string) (*grpc.ClientConn, error) { return grpcutil.GetOrCreateGRPCConn(c.ctx, &c.clientConns, url, c.tlsCfg, c.option.GRPCDialOptions...) } diff --git a/client/servicediscovery/pd_service_discovery_test.go b/client/servicediscovery/service_discovery_test.go similarity index 96% rename from client/servicediscovery/pd_service_discovery_test.go rename to client/servicediscovery/service_discovery_test.go index dc0a0bd4511..0a678718fdc 100644 --- a/client/servicediscovery/pd_service_discovery_test.go +++ b/client/servicediscovery/service_discovery_test.go @@ -193,14 +193,14 @@ func (suite *serviceClientTestSuite) TestServiceClient() { re.True(leader.IsConnectedToLeader()) re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1", "return(true)")) - follower.(*pdServiceClient).checkNetworkAvailable(suite.ctx) - leader.(*pdServiceClient).checkNetworkAvailable(suite.ctx) + follower.(*serviceClient).checkNetworkAvailable(suite.ctx) + leader.(*serviceClient).checkNetworkAvailable(suite.ctx) re.False(follower.Available()) re.False(leader.Available()) re.NoError(failpoint.Disable("github.com/tikv/pd/client/servicediscovery/unreachableNetwork1")) - follower.(*pdServiceClient).checkNetworkAvailable(suite.ctx) - leader.(*pdServiceClient).checkNetworkAvailable(suite.ctx) + follower.(*serviceClient).checkNetworkAvailable(suite.ctx) + leader.(*serviceClient).checkNetworkAvailable(suite.ctx) re.True(follower.Available()) re.True(leader.Available()) @@ -259,11 +259,11 @@ func (suite *serviceClientTestSuite) TestServiceClient() { re.False(leaderAPIClient.NeedRetry(pdErr2, nil)) re.False(followerAPIClient.Available()) re.True(leaderAPIClient.Available()) - followerAPIClient.(*pdServiceAPIClient).markAsAvailable() - leaderAPIClient.(*pdServiceAPIClient).markAsAvailable() + followerAPIClient.(*serviceAPIClient).markAsAvailable() + leaderAPIClient.(*serviceAPIClient).markAsAvailable() re.False(followerAPIClient.Available()) time.Sleep(time.Millisecond * 100) - followerAPIClient.(*pdServiceAPIClient).markAsAvailable() + followerAPIClient.(*serviceAPIClient).markAsAvailable() re.True(followerAPIClient.Available()) re.True(followerAPIClient.NeedRetry(nil, err)) @@ -278,7 +278,7 @@ func (suite *serviceClientTestSuite) TestServiceClientBalancer() { re := suite.Require() follower := suite.followerClient leader := suite.leaderClient - b := &pdServiceBalancer{} + b := &serviceBalancer{} b.set([]ServiceClient{leader, follower}) re.Equal(2, b.totalNode) @@ -400,7 +400,7 @@ func TestUpdateURLs(t *testing.T) { } return } - cli := &pdServiceDiscovery{option: opt.NewOption()} + cli := &serviceDiscovery{option: opt.NewOption()} cli.urls.Store([]string{}) cli.updateURLs(members[1:]) re.Equal(getURLs([]*pdpb.Member{members[1], members[3], members[2]}), cli.GetServiceURLs()) @@ -421,7 +421,7 @@ func TestGRPCDialOption(t *testing.T) { start := time.Now() ctx, cancel := context.WithTimeout(context.TODO(), 500*time.Millisecond) defer cancel() - cli := &pdServiceDiscovery{ + cli := &serviceDiscovery{ checkMembershipCh: make(chan struct{}, 1), ctx: ctx, cancel: cancel, diff --git a/client/servicediscovery/tso_service_discovery.go b/client/servicediscovery/tso_service_discovery.go index 1d2130db804..7734fd23107 100644 --- a/client/servicediscovery/tso_service_discovery.go +++ b/client/servicediscovery/tso_service_discovery.go @@ -126,10 +126,10 @@ func (t *tsoServerDiscovery) resetFailure() { // tsoServiceDiscovery is the service discovery client of the independent TSO service type tsoServiceDiscovery struct { - metacli metastorage.Client - apiSvcDiscovery ServiceDiscovery - clusterID uint64 - keyspaceID atomic.Uint32 + metacli metastorage.Client + serviceDiscovery ServiceDiscovery + clusterID uint64 + keyspaceID atomic.Uint32 // defaultDiscoveryKey is the etcd path used for discovering the serving endpoints of // the default keyspace group @@ -161,7 +161,7 @@ type tsoServiceDiscovery struct { // NewTSOServiceDiscovery returns a new client-side service discovery for the independent TSO service. func NewTSOServiceDiscovery( - ctx context.Context, metacli metastorage.Client, apiSvcDiscovery ServiceDiscovery, + ctx context.Context, metacli metastorage.Client, serviceDiscovery ServiceDiscovery, keyspaceID uint32, tlsCfg *tls.Config, option *opt.Option, ) ServiceDiscovery { ctx, cancel := context.WithCancel(ctx) @@ -169,8 +169,8 @@ func NewTSOServiceDiscovery( ctx: ctx, cancel: cancel, metacli: metacli, - apiSvcDiscovery: apiSvcDiscovery, - clusterID: apiSvcDiscovery.GetClusterID(), + serviceDiscovery: serviceDiscovery, + clusterID: serviceDiscovery.GetClusterID(), tlsCfg: tlsCfg, option: option, checkMembershipCh: make(chan struct{}, 1), @@ -351,7 +351,7 @@ func (c *tsoServiceDiscovery) ScheduleCheckMemberChanged() { // CheckMemberChanged Immediately check if there is any membership change among the primary/secondaries in // a primary/secondary configured cluster. func (c *tsoServiceDiscovery) CheckMemberChanged() error { - if err := c.apiSvcDiscovery.CheckMemberChanged(); err != nil { + if err := c.serviceDiscovery.CheckMemberChanged(); err != nil { log.Warn("[tso] failed to check member changed", errs.ZapError(err)) } if err := c.retry(tsoQueryRetryMaxTimes, tsoQueryRetryInterval, c.updateMember); err != nil { @@ -382,17 +382,17 @@ func (c *tsoServiceDiscovery) SetTSOLeaderURLUpdatedCallback(callback tsoLeaderU // GetServiceClient implements ServiceDiscovery func (c *tsoServiceDiscovery) GetServiceClient() ServiceClient { - return c.apiSvcDiscovery.GetServiceClient() + return c.serviceDiscovery.GetServiceClient() } // GetServiceClientByKind implements ServiceDiscovery func (c *tsoServiceDiscovery) GetServiceClientByKind(kind APIKind) ServiceClient { - return c.apiSvcDiscovery.GetServiceClientByKind(kind) + return c.serviceDiscovery.GetServiceClientByKind(kind) } // GetAllServiceClients implements ServiceDiscovery func (c *tsoServiceDiscovery) GetAllServiceClients() []ServiceClient { - return c.apiSvcDiscovery.GetAllServiceClients() + return c.serviceDiscovery.GetAllServiceClients() } // getPrimaryURL returns the primary URL. @@ -425,7 +425,7 @@ func (c *tsoServiceDiscovery) afterPrimarySwitched(oldPrimary, newPrimary string func (c *tsoServiceDiscovery) updateMember() error { // The keyspace membership or the primary serving URL of the keyspace group, to which this // keyspace belongs, might have been changed. We need to query tso servers to get the latest info. - tsoServerURL, err := c.getTSOServer(c.apiSvcDiscovery) + tsoServerURL, err := c.getTSOServer(c.serviceDiscovery) if err != nil { log.Error("[tso] failed to get tso server", errs.ZapError(err)) return err @@ -589,7 +589,7 @@ func (c *tsoServiceDiscovery) getTSOServer(sd ServiceDiscovery) (string, error) ) t := c.tsoServerDiscovery if len(t.urls) == 0 || t.failureCount == len(t.urls) { - urls, err = sd.(*pdServiceDiscovery).discoverMicroservice(tsoService) + urls, err = sd.(*serviceDiscovery).discoverMicroservice(tsoService) if err != nil { return "", err } diff --git a/cmd/pd-server/main.go b/cmd/pd-server/main.go index 165bcd2a12f..24ca46e7d5e 100644 --- a/cmd/pd-server/main.go +++ b/cmd/pd-server/main.go @@ -82,7 +82,7 @@ func NewServiceCommand() *cobra.Command { } cmd.AddCommand(NewTSOServiceCommand()) cmd.AddCommand(NewSchedulingServiceCommand()) - cmd.AddCommand(NewAPIServiceCommand()) + cmd.AddCommand(NewPDServiceCommand()) return cmd } @@ -128,12 +128,12 @@ func NewSchedulingServiceCommand() *cobra.Command { return cmd } -// NewAPIServiceCommand returns the API service command. -func NewAPIServiceCommand() *cobra.Command { +// NewPDServiceCommand returns the PD service command. +func NewPDServiceCommand() *cobra.Command { cmd := &cobra.Command{ Use: apiMode, - Short: "Run the API service", - Run: createAPIServerWrapper, + Short: "Run the PD service", + Run: createPDServiceWrapper, } addFlags(cmd) return cmd @@ -160,7 +160,7 @@ func addFlags(cmd *cobra.Command) { cmd.Flags().BoolP("force-new-cluster", "", false, "force to create a new one-member cluster") } -func createAPIServerWrapper(cmd *cobra.Command, args []string) { +func createPDServiceWrapper(cmd *cobra.Command, args []string) { start(cmd, args, cmd.CalledAs()) } @@ -219,7 +219,7 @@ func start(cmd *cobra.Command, args []string, services ...string) { defer log.Sync() memory.InitMemoryHook() if len(services) != 0 { - versioninfo.Log(server.APIServiceMode) + versioninfo.Log(server.PDServiceMode) } else { versioninfo.Log(server.PDMode) } diff --git a/pkg/mcs/registry/registry.go b/pkg/mcs/registry/registry.go index 6a01f091e52..2ffa04b1bf9 100644 --- a/pkg/mcs/registry/registry.go +++ b/pkg/mcs/registry/registry.go @@ -85,18 +85,18 @@ func (r *ServiceRegistry) InstallAllRESTHandler(srv bs.Server, h map[string]http serviceName := createServiceName(prefix, name) if l, ok := r.services[serviceName]; ok { if err := l.RegisterRESTHandler(h); err != nil { - log.Error("register restful API service failed", zap.String("prefix", prefix), zap.String("service-name", name), zap.Error(err)) + log.Error("register restful PD service failed", zap.String("prefix", prefix), zap.String("service-name", name), zap.Error(err)) } else { - log.Info("restful API service already registered", zap.String("prefix", prefix), zap.String("service-name", name)) + log.Info("restful PD service already registered", zap.String("prefix", prefix), zap.String("service-name", name)) } continue } l := builder(srv) r.services[serviceName] = l if err := l.RegisterRESTHandler(h); err != nil { - log.Error("register restful API service failed", zap.String("prefix", prefix), zap.String("service-name", name), zap.Error(err)) + log.Error("register restful PD service failed", zap.String("prefix", prefix), zap.String("service-name", name), zap.Error(err)) } else { - log.Info("restful API service registered successfully", zap.String("prefix", prefix), zap.String("service-name", name)) + log.Info("restful PD service registered successfully", zap.String("prefix", prefix), zap.String("service-name", name)) } } } diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 5c7166fba09..6f80572673c 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -55,7 +55,7 @@ type Cluster struct { storage storage.Storage coordinator *schedule.Coordinator checkMembershipCh chan struct{} - apiServerLeader atomic.Value + pdLeader atomic.Value running atomic.Bool // heartbeatRunner is used to process the subtree update task asynchronously. @@ -227,7 +227,7 @@ func (c *Cluster) GetStoreConfig() sc.StoreConfigProvider { return c.persistConf // AllocID allocates a new ID. func (c *Cluster) AllocID() (uint64, error) { - client, err := c.getAPIServerLeaderClient() + client, err := c.getPDLeaderClient() if err != nil { return 0, err } @@ -241,11 +241,11 @@ func (c *Cluster) AllocID() (uint64, error) { return resp.GetId(), nil } -func (c *Cluster) getAPIServerLeaderClient() (pdpb.PDClient, error) { - cli := c.apiServerLeader.Load() +func (c *Cluster) getPDLeaderClient() (pdpb.PDClient, error) { + cli := c.pdLeader.Load() if cli == nil { c.triggerMembershipCheck() - return nil, errors.New("API server leader is not found") + return nil, errors.New("PD leader is not found") } return cli.(pdpb.PDClient), nil } @@ -257,10 +257,10 @@ func (c *Cluster) triggerMembershipCheck() { } } -// SwitchAPIServerLeader switches the API server leader. -func (c *Cluster) SwitchAPIServerLeader(new pdpb.PDClient) bool { - old := c.apiServerLeader.Load() - return c.apiServerLeader.CompareAndSwap(old, new) +// SwitchPDServiceLeader switches the PD service leader. +func (c *Cluster) SwitchPDServiceLeader(new pdpb.PDClient) bool { + old := c.pdLeader.Load() + return c.pdLeader.CompareAndSwap(old, new) } func trySend(notifier chan struct{}) { diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index 784d1f45a82..413a6c601cc 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -243,7 +243,7 @@ func NewPersistConfig(cfg *Config, ttl *cache.TTLString) *PersistConfig { o.SetClusterVersion(&cfg.ClusterVersion) o.schedule.Store(&cfg.Schedule) o.replication.Store(&cfg.Replication) - // storeConfig will be fetched from TiKV by PD API server, + // storeConfig will be fetched from TiKV by PD service, // so we just set an empty value here first. o.storeConfig.Store(&sc.StoreConfig{}) o.ttl = ttl @@ -748,11 +748,11 @@ func (o *PersistConfig) IsRaftKV2() bool { // TODO: implement the following methods // AddSchedulerCfg adds the scheduler configurations. -// This method is a no-op since we only use configurations derived from one-way synchronization from API server now. +// This method is a no-op since we only use configurations derived from one-way synchronization from PD service now. func (*PersistConfig) AddSchedulerCfg(types.CheckerSchedulerType, []string) {} // RemoveSchedulerCfg removes the scheduler configurations. -// This method is a no-op since we only use configurations derived from one-way synchronization from API server now. +// This method is a no-op since we only use configurations derived from one-way synchronization from PD service now. func (*PersistConfig) RemoveSchedulerCfg(types.CheckerSchedulerType) {} // CheckLabelProperty checks if the label property is satisfied. diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index f499a0d7d50..9db2d47d0f4 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -36,7 +36,7 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" ) -// Watcher is used to watch the PD API server for any configuration changes. +// Watcher is used to watch the PD service for any configuration changes. type Watcher struct { wg sync.WaitGroup ctx context.Context @@ -76,7 +76,7 @@ type persistedConfig struct { Store sc.StoreConfig `json:"store"` } -// NewWatcher creates a new watcher to watch the config meta change from PD API server. +// NewWatcher creates a new watcher to watch the config meta change from PD service. func NewWatcher( ctx context.Context, etcdClient *clientv3.Client, diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index 3d1183bf734..bd2cc40c21d 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -159,7 +159,7 @@ func (s *Service) RegionHeartbeat(stream schedulingpb.Scheduling_RegionHeartbeat region := core.RegionFromHeartbeat(request, 0) err = c.HandleRegionHeartbeat(region) if err != nil { - // TODO: if we need to send the error back to API server. + // TODO: if we need to send the error back to PD service. log.Error("failed handle region heartbeat", zap.Error(err)) continue } diff --git a/pkg/mcs/scheduling/server/meta/watcher.go b/pkg/mcs/scheduling/server/meta/watcher.go index 27fe6687f3d..c51f10027d7 100644 --- a/pkg/mcs/scheduling/server/meta/watcher.go +++ b/pkg/mcs/scheduling/server/meta/watcher.go @@ -33,7 +33,7 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" ) -// Watcher is used to watch the PD API server for any meta changes. +// Watcher is used to watch the PD service for any meta changes. type Watcher struct { wg sync.WaitGroup ctx context.Context @@ -48,7 +48,7 @@ type Watcher struct { storeWatcher *etcdutil.LoopWatcher } -// NewWatcher creates a new watcher to watch the meta change from PD API server. +// NewWatcher creates a new watcher to watch the meta change from PD service. func NewWatcher( ctx context.Context, etcdClient *clientv3.Client, diff --git a/pkg/mcs/scheduling/server/rule/watcher.go b/pkg/mcs/scheduling/server/rule/watcher.go index cc6480a0cb4..014a3abc2be 100644 --- a/pkg/mcs/scheduling/server/rule/watcher.go +++ b/pkg/mcs/scheduling/server/rule/watcher.go @@ -34,7 +34,7 @@ import ( "github.com/tikv/pd/pkg/utils/keypath" ) -// Watcher is used to watch the PD API server for any Placement Rule changes. +// Watcher is used to watch the PD service for any Placement Rule changes. type Watcher struct { ctx context.Context cancel context.CancelFunc @@ -74,7 +74,7 @@ type Watcher struct { patch *placement.RuleConfigPatch } -// NewWatcher creates a new watcher to watch the Placement Rule change from PD API server. +// NewWatcher creates a new watcher to watch the Placement Rule change from PD service. func NewWatcher( ctx context.Context, etcdClient *clientv3.Client, diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index 8c9972d5eec..80156c1e26b 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -110,7 +110,7 @@ type Server struct { hbStreams *hbstream.HeartbeatStreams storage *endpoint.StorageEndpoint - // for watching the PD API server meta info updates that are related to the scheduling. + // for watching the PD service meta info updates that are related to the scheduling. configWatcher *config.Watcher ruleWatcher *rule.Watcher metaWatcher *meta.Watcher @@ -169,10 +169,10 @@ func (s *Server) startServerLoop() { s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.Context()) s.serverLoopWg.Add(2) go s.primaryElectionLoop() - go s.updateAPIServerMemberLoop() + go s.updatePDServiceMemberLoop() } -func (s *Server) updateAPIServerMemberLoop() { +func (s *Server) updatePDServiceMemberLoop() { defer logutil.LogPanic() defer s.serverLoopWg.Done() @@ -220,7 +220,7 @@ func (s *Server) updateAPIServerMemberLoop() { // double check break } - if s.cluster.SwitchAPIServerLeader(pdpb.NewPDClient(cc)) { + if s.cluster.SwitchPDServiceLeader(pdpb.NewPDClient(cc)) { if status.Leader != curLeader { log.Info("switch leader", zap.String("leader-id", fmt.Sprintf("%x", ep.ID)), zap.String("endpoint", ep.ClientURLs[0])) } diff --git a/pkg/mcs/utils/constant/constant.go b/pkg/mcs/utils/constant/constant.go index 87fcf29f678..6f684bdb977 100644 --- a/pkg/mcs/utils/constant/constant.go +++ b/pkg/mcs/utils/constant/constant.go @@ -57,8 +57,8 @@ const ( // MicroserviceRootPath is the root path of microservice in etcd. MicroserviceRootPath = "/ms" - // APIServiceName is the name of api server. - APIServiceName = "api" + // PDServiceName is the name of pd server. + PDServiceName = "pd" // TSOServiceName is the name of tso server. TSOServiceName = "tso" // SchedulingServiceName is the name of scheduling server. diff --git a/pkg/member/election_leader.go b/pkg/member/election_leader.go index 81afc5dbd0a..2e5769d7dc4 100644 --- a/pkg/member/election_leader.go +++ b/pkg/member/election_leader.go @@ -21,7 +21,7 @@ import ( ) // ElectionLeader defines the common interface of the leader, which is the pdpb.Member -// for in PD/API service or the tsopb.Participant in the micro services. +// for in PD/PD service or the tsopb.Participant in the micro services. type ElectionLeader interface { // GetListenUrls returns the listen urls GetListenUrls() []string diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index e792560cb37..80299bf1e25 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -389,7 +389,7 @@ func (c *Coordinator) LoadPlugin(pluginPath string, ch chan string) { return } log.Info("create scheduler", zap.String("scheduler-name", s.GetName())) - // TODO: handle the plugin in API service mode. + // TODO: handle the plugin in PD service mode. if err = c.schedulers.AddScheduler(s); err != nil { log.Error("can't add scheduler", zap.String("scheduler-name", s.GetName()), errs.ZapError(err)) return diff --git a/pkg/schedule/schedulers/scheduler_controller.go b/pkg/schedule/schedulers/scheduler_controller.go index 28973631570..5f461d326c5 100644 --- a/pkg/schedule/schedulers/scheduler_controller.go +++ b/pkg/schedule/schedulers/scheduler_controller.go @@ -55,7 +55,7 @@ type Controller struct { // and used in the PD leader service mode now. schedulers map[string]*ScheduleController // schedulerHandlers is used to manage the HTTP handlers of schedulers, - // which will only be initialized and used in the API service mode now. + // which will only be initialized and used in the PD service mode now. schedulerHandlers map[string]http.Handler opController *operator.Controller } diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 149c68029be..9793939fa17 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -334,7 +334,7 @@ type KeyspaceGroupManager struct { // Value: discover.ServiceRegistryEntry tsoServiceKey string // legacySvcRootPath defines the legacy root path for all etcd paths which derives from - // the PD/API service. It's in the format of "/pd/{cluster_id}". + // the PD/PD service. It's in the format of "/pd/{cluster_id}". // The main paths for different usages include: // 1. The path, used by the default keyspace group, for LoadTimestamp/SaveTimestamp in the // storage endpoint. diff --git a/pkg/utils/apiutil/serverapi/middleware.go b/pkg/utils/apiutil/serverapi/middleware.go index 85b958a5554..823deed64ea 100644 --- a/pkg/utils/apiutil/serverapi/middleware.go +++ b/pkg/utils/apiutil/serverapi/middleware.go @@ -116,7 +116,7 @@ func MicroserviceRedirectRule(matchPath, targetPath, targetServiceName string, } func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, string) { - if !h.s.IsAPIServiceMode() { + if !h.s.IsPDServiceMode() { return false, "" } if len(h.microserviceRedirectRules) == 0 { @@ -223,7 +223,7 @@ func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http clientUrls = leader.GetClientUrls() r.Header.Set(apiutil.PDRedirectorHeader, h.s.Name()) } else { - // Prevent more than one redirection among PD/API servers. + // Prevent more than one redirection among PD/PD service. log.Error("redirect but server is not leader", zap.String("from", name), zap.String("server", h.s.Name()), errs.ZapError(errs.ErrRedirectToNotLeader)) http.Error(w, errs.ErrRedirectToNotLeader.FastGenByArgs().Error(), http.StatusInternalServerError) return diff --git a/server/api/admin.go b/server/api/admin.go index d2be53cf40e..561f4ec4bff 100644 --- a/server/api/admin.go +++ b/server/api/admin.go @@ -254,5 +254,5 @@ func (h *adminHandler) deleteRegionCacheInSchedulingServer(id ...uint64) error { } func buildMsg(err error) string { - return fmt.Sprintf("This operation was executed in API server but needs to be re-executed on scheduling server due to the following error: %s", err.Error()) + return fmt.Sprintf("This operation was executed in PD service but needs to be re-executed on scheduling server due to the following error: %s", err.Error()) } diff --git a/server/api/server.go b/server/api/server.go index 1a744635e2d..8e352b6a36e 100644 --- a/server/api/server.go +++ b/server/api/server.go @@ -63,7 +63,7 @@ func NewHandler(_ context.Context, svr *server.Server) (http.Handler, apiutil.AP // Following requests are **not** redirected: // "/schedulers", http.MethodPost // "/schedulers/{name}", http.MethodDelete - // Because the writing of all the config of the scheduling service is in the API server, + // Because the writing of all the config of the scheduling service is in the PD service, // we should not post and delete the scheduler directly in the scheduling service. router.PathPrefix(apiPrefix).Handler(negroni.New( serverapi.NewRuntimeServiceValidator(svr, group), @@ -153,7 +153,7 @@ func NewHandler(_ context.Context, svr *server.Server) (http.Handler, apiutil.AP scheapi.APIPathPrefix+"/config/placement-rule", constant.SchedulingServiceName, []string{http.MethodGet}), - // because the writing of all the meta information of the scheduling service is in the API server, + // because the writing of all the meta information of the scheduling service is in the PD service, // we should not post and delete the scheduler directly in the scheduling service. serverapi.MicroserviceRedirectRule( prefix+"/schedulers", diff --git a/server/apiv2/handlers/micro_service.go b/server/apiv2/handlers/micro_service.go index c7fa0dc94f2..b4d3d6bbe89 100644 --- a/server/apiv2/handlers/micro_service.go +++ b/server/apiv2/handlers/micro_service.go @@ -39,7 +39,7 @@ func RegisterMicroService(r *gin.RouterGroup) { // @Router /ms/members/{service} [get] func GetMembers(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) - if !svr.IsAPIServiceMode() { + if !svr.IsPDServiceMode() { c.AbortWithStatusJSON(http.StatusNotFound, "not support micro service") return } @@ -65,7 +65,7 @@ func GetMembers(c *gin.Context) { // @Router /ms/primary/{service} [get] func GetPrimary(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) - if !svr.IsAPIServiceMode() { + if !svr.IsPDServiceMode() { c.AbortWithStatusJSON(http.StatusNotFound, "not support micro service") return } diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index d2f3855d14e..e482b7b8b68 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -131,7 +131,7 @@ type Server interface { GetMembers() ([]*pdpb.Member, error) ReplicateFileToMember(ctx context.Context, member *pdpb.Member, name string, data []byte) error GetKeyspaceGroupManager() *keyspace.GroupManager - IsAPIServiceMode() bool + IsPDServiceMode() bool GetSafePointV2Manager() *gc.SafePointV2Manager } @@ -156,12 +156,12 @@ type RaftCluster struct { etcdClient *clientv3.Client httpClient *http.Client - running bool - isAPIServiceMode bool - meta *metapb.Cluster - storage storage.Storage - minResolvedTS atomic.Value // Store as uint64 - externalTS atomic.Value // Store as uint64 + running bool + isPDServiceMode bool + meta *metapb.Cluster + storage storage.Storage + minResolvedTS atomic.Value // Store as uint64 + externalTS atomic.Value // Store as uint64 // Keep the previous store limit settings when removing a store. prevStoreLimit map[uint64]map[storelimit.Type]float64 @@ -325,7 +325,7 @@ func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { log.Warn("raft cluster has already been started") return nil } - c.isAPIServiceMode = s.IsAPIServiceMode() + c.isPDServiceMode = s.IsPDServiceMode() err = c.InitCluster(s.GetAllocator(), s.GetPersistOptions(), s.GetHBStreams(), s.GetKeyspaceGroupManager()) if err != nil { return err @@ -376,7 +376,7 @@ func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { c.loadExternalTS() c.loadMinResolvedTS() - if c.isAPIServiceMode { + if c.isPDServiceMode { // bootstrap keyspace group manager after starting other parts successfully. // This order avoids a stuck goroutine in keyspaceGroupManager when it fails to create raftcluster. err = c.keyspaceGroupManager.Bootstrap(c.ctx) @@ -404,7 +404,7 @@ func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { } func (c *RaftCluster) checkSchedulingService() { - if c.isAPIServiceMode { + if c.isPDServiceMode { servers, err := discovery.Discover(c.etcdClient, constant.SchedulingServiceName) if c.opt.GetMicroServiceConfig().IsSchedulingFallbackEnabled() && (err != nil || len(servers) == 0) { c.startSchedulingJobs(c, c.hbstreams) @@ -425,7 +425,7 @@ func (c *RaftCluster) checkSchedulingService() { // checkTSOService checks the TSO service. func (c *RaftCluster) checkTSOService() { - if c.isAPIServiceMode { + if c.isPDServiceMode { if c.opt.GetMicroServiceConfig().IsTSODynamicSwitchingEnabled() { servers, err := discovery.Discover(c.etcdClient, constant.TSOServiceName) if err != nil || len(servers) == 0 { diff --git a/server/config/config.go b/server/config/config.go index 282b5264fe9..69cd76409bc 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -860,7 +860,7 @@ func (c *MicroServiceConfig) Clone() *MicroServiceConfig { return &cfg } -// IsSchedulingFallbackEnabled returns whether to enable scheduling service fallback to api service. +// IsSchedulingFallbackEnabled returns whether to enable scheduling service fallback to PD service. func (c *MicroServiceConfig) IsSchedulingFallbackEnabled() bool { return c.EnableSchedulingFallback } diff --git a/server/grpc_service.go b/server/grpc_service.go index d3fc5c58d7f..118b3d84748 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -271,8 +271,8 @@ func (s *GrpcServer) GetClusterInfo(context.Context, *pdpb.GetClusterInfoRequest }, nil } -// GetMinTS implements gRPC PDServer. In PD service mode, it simply returns a timestamp. -// In API service mode, it queries all tso servers and gets the minimum timestamp across +// GetMinTS implements gRPC PDServer. In PD mode, it simply returns a timestamp. +// In PD service mode, it queries all tso servers and gets the minimum timestamp across // all keyspace groups. func (s *GrpcServer) GetMinTS( ctx context.Context, request *pdpb.GetMinTSRequest, diff --git a/server/server.go b/server/server.go index 94250128fe3..8de2a5c57f3 100644 --- a/server/server.go +++ b/server/server.go @@ -101,8 +101,8 @@ const ( // PDMode represents that server is in PD mode. PDMode = "PD" - // APIServiceMode represents that server is in API service mode. - APIServiceMode = "API Service" + // PDServiceMode represents that server is in PD service mode which is in microservice architecture. + PDServiceMode = "PD Service" // maxRetryTimesGetServicePrimary is the max retry times for getting primary addr. // Note: it need to be less than client.defaultPDTimeout @@ -243,7 +243,7 @@ type HandlerBuilder func(context.Context, *Server) (http.Handler, apiutil.APISer func CreateServer(ctx context.Context, cfg *config.Config, services []string, legacyServiceBuilders ...HandlerBuilder) (*Server, error) { var mode string if len(services) != 0 { - mode = APIServiceMode + mode = PDServiceMode } else { mode = PDMode } @@ -478,7 +478,7 @@ func (s *Server) startServer(ctx context.Context) error { Member: s.member.MemberValue(), Step: keyspace.AllocStep, }) - if s.IsAPIServiceMode() { + if s.IsPDServiceMode() { s.keyspaceGroupManager = keyspace.NewKeyspaceGroupManager(s.ctx, s.storage, s.client) } s.keyspaceManager = keyspace.NewKeyspaceManager(s.ctx, s.storage, s.cluster, keyspaceIDAllocator, &s.cfg.Keyspace, s.keyspaceGroupManager) @@ -530,7 +530,7 @@ func (s *Server) Close() { s.cgMonitor.StopMonitor() s.stopServerLoop() - if s.IsAPIServiceMode() { + if s.IsPDServiceMode() { s.keyspaceGroupManager.Close() } @@ -641,7 +641,7 @@ func (s *Server) startServerLoop(ctx context.Context) { go s.etcdLeaderLoop() go s.serverMetricsLoop() go s.encryptionKeyManagerLoop() - if s.IsAPIServiceMode() { + if s.IsPDServiceMode() { s.initTSOPrimaryWatcher() s.initSchedulingPrimaryWatcher() } @@ -788,9 +788,9 @@ func (s *Server) stopRaftCluster() { s.cluster.Stop() } -// IsAPIServiceMode return whether the server is in API service mode. -func (s *Server) IsAPIServiceMode() bool { - return s.mode == APIServiceMode +// IsPDServiceMode return whether the server is in PD service mode. +func (s *Server) IsPDServiceMode() bool { + return s.mode == PDServiceMode } // GetAddr returns the server urls for clients. @@ -1390,7 +1390,7 @@ func (s *Server) GetRaftCluster() *cluster.RaftCluster { // IsServiceIndependent returns whether the service is independent. func (s *Server) IsServiceIndependent(name string) bool { - if s.mode == APIServiceMode && !s.IsClosed() { + if s.mode == PDServiceMode && !s.IsClosed() { if name == constant.TSOServiceName && !s.GetMicroServiceConfig().IsTSODynamicSwitchingEnabled() { return true } @@ -1667,7 +1667,7 @@ func (s *Server) campaignLeader() { log.Info(fmt.Sprintf("start to campaign %s leader", s.mode), zap.String("campaign-leader-name", s.Name())) if err := s.member.CampaignLeader(s.ctx, s.cfg.LeaderLease); err != nil { if err.Error() == errs.ErrEtcdTxnConflict.Error() { - log.Info(fmt.Sprintf("campaign %s leader meets error due to txn conflict, another PD/API server may campaign successfully", s.mode), + log.Info(fmt.Sprintf("campaign %s leader meets error due to txn conflict, another PD/PD service may campaign successfully", s.mode), zap.String("campaign-leader-name", s.Name())) } else { log.Error(fmt.Sprintf("campaign %s leader meets error due to etcd error", s.mode), diff --git a/server/server_test.go b/server/server_test.go index 23da2078cb2..28839b89389 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -266,13 +266,13 @@ func TestAPIService(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() mockHandler := CreateMockHandler(re, "127.0.0.1") - svr, err := CreateServer(ctx, cfg, []string{constant.APIServiceName}, mockHandler) + svr, err := CreateServer(ctx, cfg, []string{constant.PDServiceName}, mockHandler) re.NoError(err) defer svr.Close() err = svr.Run() re.NoError(err) MustWaitLeader(re, []*Server{svr}) - re.True(svr.IsAPIServiceMode()) + re.True(svr.IsPDServiceMode()) } func TestIsPathInDirectory(t *testing.T) { diff --git a/tests/cluster.go b/tests/cluster.go index a4f445155e1..4189b43902a 100644 --- a/tests/cluster.go +++ b/tests/cluster.go @@ -79,16 +79,7 @@ type TestServer struct { var zapLogOnce sync.Once // NewTestServer creates a new TestServer. -func NewTestServer(ctx context.Context, cfg *config.Config) (*TestServer, error) { - return createTestServer(ctx, cfg, nil) -} - -// NewTestAPIServer creates a new TestServer. -func NewTestAPIServer(ctx context.Context, cfg *config.Config) (*TestServer, error) { - return createTestServer(ctx, cfg, []string{constant.APIServiceName}) -} - -func createTestServer(ctx context.Context, cfg *config.Config, services []string) (*TestServer, error) { +func NewTestServer(ctx context.Context, cfg *config.Config, services []string) (*TestServer, error) { // disable the heartbeat async runner in test cfg.Schedule.EnableHeartbeatConcurrentRunner = false err := logutil.SetupLogger(cfg.Log, &cfg.Logger, &cfg.LogProps, cfg.Security.RedactInfoLog) @@ -435,15 +426,15 @@ type ConfigOption func(conf *config.Config, serverName string) // NewTestCluster creates a new TestCluster. func NewTestCluster(ctx context.Context, initialServerCount int, opts ...ConfigOption) (*TestCluster, error) { - return createTestCluster(ctx, initialServerCount, false, opts...) + return createTestCluster(ctx, initialServerCount, nil, opts...) } -// NewTestAPICluster creates a new TestCluster with API service. -func NewTestAPICluster(ctx context.Context, initialServerCount int, opts ...ConfigOption) (*TestCluster, error) { - return createTestCluster(ctx, initialServerCount, true, opts...) +// NewTestPDServiceCluster creates a new TestCluster with PD service. +func NewTestPDServiceCluster(ctx context.Context, initialServerCount int, opts ...ConfigOption) (*TestCluster, error) { + return createTestCluster(ctx, initialServerCount, []string{constant.PDServiceName}, opts...) } -func createTestCluster(ctx context.Context, initialServerCount int, isAPIServiceMode bool, opts ...ConfigOption) (*TestCluster, error) { +func createTestCluster(ctx context.Context, initialServerCount int, services []string, opts ...ConfigOption) (*TestCluster, error) { schedulers.Register() config := newClusterConfig(initialServerCount) servers := make(map[string]*TestServer) @@ -452,12 +443,7 @@ func createTestCluster(ctx context.Context, initialServerCount int, isAPIService if err != nil { return nil, err } - var s *TestServer - if isAPIServiceMode { - s, err = NewTestAPIServer(ctx, serverConf) - } else { - s, err = NewTestServer(ctx, serverConf) - } + s, err := NewTestServer(ctx, serverConf, services) if err != nil { return nil, err } @@ -481,7 +467,7 @@ func RestartTestAPICluster(ctx context.Context, cluster *TestCluster) (*TestClus } func restartTestCluster( - ctx context.Context, cluster *TestCluster, isAPIServiceMode bool, + ctx context.Context, cluster *TestCluster, isPDServiceMode bool, ) (newTestCluster *TestCluster, err error) { schedulers.Register() newTestCluster = &TestCluster{ @@ -508,10 +494,10 @@ func restartTestCluster( newServer *TestServer serverErr error ) - if isAPIServiceMode { - newServer, serverErr = NewTestAPIServer(ctx, serverCfg) + if isPDServiceMode { + newServer, serverErr = NewTestServer(ctx, serverCfg, []string{constant.PDServiceName}) } else { - newServer, serverErr = NewTestServer(ctx, serverCfg) + newServer, serverErr = NewTestServer(ctx, serverCfg, nil) } serverMap.Store(serverName, newServer) errorMap.Store(serverName, serverErr) @@ -735,7 +721,7 @@ func (c *TestCluster) Join(ctx context.Context, opts ...ConfigOption) (*TestServ if err != nil { return nil, err } - s, err := NewTestServer(ctx, conf) + s, err := NewTestServer(ctx, conf, nil) if err != nil { return nil, err } @@ -743,13 +729,13 @@ func (c *TestCluster) Join(ctx context.Context, opts ...ConfigOption) (*TestServ return s, nil } -// JoinAPIServer is used to add a new TestAPIServer into the cluster. -func (c *TestCluster) JoinAPIServer(ctx context.Context, opts ...ConfigOption) (*TestServer, error) { +// JoinPDServer is used to add a new TestServer into the cluster. +func (c *TestCluster) JoinPDServer(ctx context.Context, opts ...ConfigOption) (*TestServer, error) { conf, err := c.config.join().Generate(opts...) if err != nil { return nil, err } - s, err := NewTestAPIServer(ctx, conf) + s, err := NewTestServer(ctx, conf, []string{constant.PDServiceName}) if err != nil { return nil, err } diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index 397e1079af3..ab3874a33a7 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -361,7 +361,7 @@ func TestTSOFollowerProxyWithTSOService(t *testing.T) { re.NoError(failpoint.Enable("github.com/tikv/pd/client/servicediscovery/fastUpdateServiceMode", `return(true)`)) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - cluster, err := tests.NewTestAPICluster(ctx, 1) + cluster, err := tests.NewTestPDServiceCluster(ctx, 1) re.NoError(err) defer cluster.Destroy() err = cluster.RunInitialServers() diff --git a/tests/integrations/mcs/discovery/register_test.go b/tests/integrations/mcs/discovery/register_test.go index da6fa158307..eb8933e10d8 100644 --- a/tests/integrations/mcs/discovery/register_test.go +++ b/tests/integrations/mcs/discovery/register_test.go @@ -54,7 +54,7 @@ func (suite *serverRegisterTestSuite) SetupSuite() { re := suite.Require() suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() @@ -84,7 +84,7 @@ func (suite *serverRegisterTestSuite) checkServerRegister(serviceName string) { addr := s.GetAddr() client := suite.pdLeader.GetEtcdClient() - // test API server discovery + // test PD service discovery endpoints, err := discovery.Discover(client, serviceName) re.NoError(err) @@ -98,7 +98,7 @@ func (suite *serverRegisterTestSuite) checkServerRegister(serviceName string) { re.True(exist) re.Equal(expectedPrimary, primary) - // test API server discovery after unregister + // test PD service discovery after unregister cleanup() endpoints, err = discovery.Discover(client, serviceName) re.NoError(err) @@ -140,7 +140,7 @@ func (suite *serverRegisterTestSuite) checkServerPrimaryChange(serviceName strin delete(serverMap, primary) expectedPrimary = tests.WaitForPrimaryServing(re, serverMap) - // test API server discovery + // test PD service discovery client := suite.pdLeader.GetEtcdClient() endpoints, err := discovery.Discover(client, serviceName) re.NoError(err) diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index 44347b4757d..b31d919324d 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -60,7 +60,7 @@ func (suite *keyspaceGroupTestSuite) SetupTest() { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) ctx, cancel := context.WithCancel(context.Background()) suite.ctx = ctx - cluster, err := tests.NewTestAPICluster(suite.ctx, 1) + cluster, err := tests.NewTestPDServiceCluster(suite.ctx, 1) suite.cluster = cluster re.NoError(err) re.NoError(cluster.RunInitialServers()) diff --git a/tests/integrations/mcs/members/member_test.go b/tests/integrations/mcs/members/member_test.go index 28275849073..7e83ea570b9 100644 --- a/tests/integrations/mcs/members/member_test.go +++ b/tests/integrations/mcs/members/member_test.go @@ -64,7 +64,7 @@ func (suite *memberTestSuite) SetupTest() { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) ctx, cancel := context.WithCancel(context.Background()) suite.ctx = ctx - cluster, err := tests.NewTestAPICluster(suite.ctx, 1) + cluster, err := tests.NewTestPDServiceCluster(suite.ctx, 1) suite.cluster = cluster re.NoError(err) re.NoError(cluster.RunInitialServers()) diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index abace06bb78..9b6b3d95145 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -56,7 +56,7 @@ func (suite *apiTestSuite) TearDownSuite() { } func (suite *apiTestSuite) TestGetCheckerByName() { - suite.env.RunTestInAPIMode(suite.checkGetCheckerByName) + suite.env.RunTestInPDServiceMode(suite.checkGetCheckerByName) } func (suite *apiTestSuite) checkGetCheckerByName(cluster *tests.TestCluster) { @@ -102,7 +102,7 @@ func (suite *apiTestSuite) checkGetCheckerByName(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestAPIForward() { - suite.env.RunTestInAPIMode(suite.checkAPIForward) + suite.env.RunTestInPDServiceMode(suite.checkAPIForward) } func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { @@ -378,7 +378,7 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestConfig() { - suite.env.RunTestInAPIMode(suite.checkConfig) + suite.env.RunTestInPDServiceMode(suite.checkConfig) } func (suite *apiTestSuite) checkConfig(cluster *tests.TestCluster) { @@ -401,7 +401,7 @@ func (suite *apiTestSuite) checkConfig(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestConfigForward() { - suite.env.RunTestInAPIMode(suite.checkConfigForward) + suite.env.RunTestInPDServiceMode(suite.checkConfigForward) } func (suite *apiTestSuite) checkConfigForward(cluster *tests.TestCluster) { @@ -413,7 +413,7 @@ func (suite *apiTestSuite) checkConfigForward(cluster *tests.TestCluster) { urlPrefix := fmt.Sprintf("%s/pd/api/v1/config", addr) // Test config forward - // Expect to get same config in scheduling server and api server + // Expect to get same config in scheduling server and PD service testutil.Eventually(re, func() bool { testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &cfg) re.Equal(cfg["schedule"].(map[string]any)["leader-schedule-limit"], @@ -421,8 +421,8 @@ func (suite *apiTestSuite) checkConfigForward(cluster *tests.TestCluster) { return cfg["replication"].(map[string]any)["max-replicas"] == float64(opts.GetReplicationConfig().MaxReplicas) }) - // Test to change config in api server - // Expect to get new config in scheduling server and api server + // Test to change config in PD service + // Expect to get new config in scheduling server and PD service reqData, err := json.Marshal(map[string]any{ "max-replicas": 4, }) @@ -436,7 +436,7 @@ 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 + // Expect to get new config in scheduling server but not old config in PD service scheCfg := opts.GetScheduleConfig().Clone() scheCfg.LeaderScheduleLimit = 100 opts.SetScheduleConfig(scheCfg) @@ -452,7 +452,7 @@ func (suite *apiTestSuite) checkConfigForward(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestAdminRegionCache() { - suite.env.RunTestInAPIMode(suite.checkAdminRegionCache) + suite.env.RunTestInPDServiceMode(suite.checkAdminRegionCache) } func (suite *apiTestSuite) checkAdminRegionCache(cluster *tests.TestCluster) { @@ -479,7 +479,7 @@ func (suite *apiTestSuite) checkAdminRegionCache(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestAdminRegionCacheForward() { - suite.env.RunTestInAPIMode(suite.checkAdminRegionCacheForward) + suite.env.RunTestInPDServiceMode(suite.checkAdminRegionCacheForward) } func (suite *apiTestSuite) checkAdminRegionCacheForward(cluster *tests.TestCluster) { @@ -491,22 +491,22 @@ func (suite *apiTestSuite) checkAdminRegionCacheForward(cluster *tests.TestClust r3 := core.NewTestRegionInfo(30, 1, []byte("c"), []byte(""), core.SetRegionConfVer(100), core.SetRegionVersion(100)) tests.MustPutRegionInfo(re, cluster, r3) - apiServer := cluster.GetLeaderServer().GetServer() + pdServer := cluster.GetLeaderServer().GetServer() schedulingServer := cluster.GetSchedulingPrimaryServer() re.Equal(3, schedulingServer.GetCluster().GetRegionCount([]byte{}, []byte{})) - re.Equal(3, apiServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) + re.Equal(3, pdServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) addr := cluster.GetLeaderServer().GetAddr() urlPrefix := fmt.Sprintf("%s/pd/api/v1/admin/cache/region", addr) err := testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "30"), testutil.StatusOK(re)) re.NoError(err) re.Equal(2, schedulingServer.GetCluster().GetRegionCount([]byte{}, []byte{})) - re.Equal(2, apiServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) + re.Equal(2, pdServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) err = testutil.CheckDelete(tests.TestDialClient, urlPrefix+"s", testutil.StatusOK(re)) re.NoError(err) re.Equal(0, schedulingServer.GetCluster().GetRegionCount([]byte{}, []byte{})) - re.Equal(0, apiServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) + re.Equal(0, pdServer.GetRaftCluster().GetRegionCount([]byte{}, []byte{})) } func (suite *apiTestSuite) TestFollowerForward() { @@ -520,7 +520,7 @@ func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { leaderAddr := cluster.GetLeaderServer().GetAddr() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - follower, err := cluster.JoinAPIServer(ctx) + follower, err := cluster.JoinPDServer(ctx) re.NoError(err) re.NoError(follower.Run()) re.NotEmpty(cluster.WaitLeader()) @@ -558,7 +558,7 @@ func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestMetrics() { - suite.env.RunTestInAPIMode(suite.checkMetrics) + suite.env.RunTestInPDServiceMode(suite.checkMetrics) } func (suite *apiTestSuite) checkMetrics(cluster *tests.TestCluster) { @@ -577,7 +577,7 @@ func (suite *apiTestSuite) checkMetrics(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestStatus() { - suite.env.RunTestInAPIMode(suite.checkStatus) + suite.env.RunTestInPDServiceMode(suite.checkStatus) } func (suite *apiTestSuite) checkStatus(cluster *tests.TestCluster) { @@ -600,7 +600,7 @@ func (suite *apiTestSuite) checkStatus(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestStores() { - suite.env.RunTestInAPIMode(suite.checkStores) + suite.env.RunTestInPDServiceMode(suite.checkStores) } func (suite *apiTestSuite) checkStores(cluster *tests.TestCluster) { @@ -647,8 +647,8 @@ func (suite *apiTestSuite) checkStores(cluster *tests.TestCluster) { tests.MustPutStore(re, cluster, store) } // Test /stores - apiServerAddr := cluster.GetLeaderServer().GetAddr() - urlPrefix := fmt.Sprintf("%s/pd/api/v1/stores", apiServerAddr) + pdServiceAddr := cluster.GetLeaderServer().GetAddr() + urlPrefix := fmt.Sprintf("%s/pd/api/v1/stores", pdServiceAddr) var resp map[string]any err := testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &resp) re.NoError(err) @@ -682,7 +682,7 @@ func (suite *apiTestSuite) checkStores(cluster *tests.TestCluster) { } func (suite *apiTestSuite) TestRegions() { - suite.env.RunTestInAPIMode(suite.checkRegions) + suite.env.RunTestInPDServiceMode(suite.checkRegions) } func (suite *apiTestSuite) checkRegions(cluster *tests.TestCluster) { @@ -691,8 +691,8 @@ func (suite *apiTestSuite) checkRegions(cluster *tests.TestCluster) { tests.MustPutRegion(re, cluster, 2, 2, []byte("c"), []byte("d")) tests.MustPutRegion(re, cluster, 3, 1, []byte("e"), []byte("f")) // Test /regions - apiServerAddr := cluster.GetLeaderServer().GetAddr() - urlPrefix := fmt.Sprintf("%s/pd/api/v1/regions", apiServerAddr) + pdServiceAddr := cluster.GetLeaderServer().GetAddr() + urlPrefix := fmt.Sprintf("%s/pd/api/v1/regions", pdServiceAddr) var resp map[string]any err := testutil.ReadGetJSON(re, tests.TestDialClient, urlPrefix, &resp) re.NoError(err) diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index d7d200814bb..6c770d3e4c1 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -62,7 +62,7 @@ func (suite *configTestSuite) SetupSuite() { schedulers.Register() var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() re.NoError(err) @@ -132,7 +132,7 @@ func (suite *configTestSuite) TestConfigWatch() { watcher.Close() } -// Manually trigger the config persistence in the PD API server side. +// Manually trigger the config persistence in the PD service side. func persistConfig(re *require.Assertions, pdLeaderServer *tests.TestServer) { err := pdLeaderServer.GetPersistOptions().Persist(pdLeaderServer.GetServer().GetStorage()) re.NoError(err) @@ -152,19 +152,19 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { ) re.NoError(err) // Get all default scheduler names. - var namesFromAPIServer []string + var namesFromPDService []string testutil.Eventually(re, func() bool { - namesFromAPIServer, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllSchedulerConfigs() - return len(namesFromAPIServer) == len(sc.DefaultSchedulers) + namesFromPDService, _, _ = suite.pdLeaderServer.GetRaftCluster().GetStorage().LoadAllSchedulerConfigs() + return len(namesFromPDService) == len(sc.DefaultSchedulers) }) // Check all default schedulers' configs. var namesFromSchedulingServer []string testutil.Eventually(re, func() bool { namesFromSchedulingServer, _, err = storage.LoadAllSchedulerConfigs() re.NoError(err) - return len(namesFromSchedulingServer) == len(namesFromAPIServer) + return len(namesFromSchedulingServer) == len(namesFromPDService) }) - re.Equal(namesFromAPIServer, namesFromSchedulingServer) + re.Equal(namesFromPDService, namesFromSchedulingServer) // Add a new scheduler. api.MustAddScheduler(re, suite.pdLeaderServer.GetAddr(), types.EvictLeaderScheduler.String(), map[string]any{ "store_id": 1, diff --git a/tests/integrations/mcs/scheduling/meta_test.go b/tests/integrations/mcs/scheduling/meta_test.go index 4e0d5249fdb..8df576b82ca 100644 --- a/tests/integrations/mcs/scheduling/meta_test.go +++ b/tests/integrations/mcs/scheduling/meta_test.go @@ -53,7 +53,7 @@ func (suite *metaTestSuite) SetupSuite() { re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs", `return(true)`)) var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() re.NoError(err) diff --git a/tests/integrations/mcs/scheduling/rule_test.go b/tests/integrations/mcs/scheduling/rule_test.go index 880dfddbb16..706c5784831 100644 --- a/tests/integrations/mcs/scheduling/rule_test.go +++ b/tests/integrations/mcs/scheduling/rule_test.go @@ -54,7 +54,7 @@ func (suite *ruleTestSuite) SetupSuite() { var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() re.NoError(err) @@ -97,7 +97,7 @@ func (suite *ruleTestSuite) TestRuleWatch() { re.Equal(placement.DefaultGroupID, ruleGroups[0].ID) re.Equal(0, ruleGroups[0].Index) re.False(ruleGroups[0].Override) - // Set a new rule via the PD API server. + // Set a new rule via the PD service. apiRuleManager := suite.pdLeaderServer.GetRaftCluster().GetRuleManager() rule := &placement.Rule{ GroupID: "2", diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index 3401fb880cb..9a3d33d1dcf 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -66,7 +66,7 @@ func (suite *serverTestSuite) SetupSuite() { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/scheduling/server/changeRunCollectWaitTime", `return(true)`)) re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs", `return(true)`)) suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() @@ -220,7 +220,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { // Change back to the default value. conf.EnableSchedulingFallback = true leaderServer.SetMicroServiceConfig(*conf) - // API server will execute scheduling jobs since there is no scheduling server. + // PD service will execute scheduling jobs since there is no scheduling server. testutil.Eventually(re, func() bool { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -229,7 +229,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) - // After scheduling server is started, API server will not execute scheduling jobs. + // After scheduling server is started, PD service will not execute scheduling jobs. testutil.Eventually(re, func() bool { return !suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -238,7 +238,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { return tc.GetPrimaryServer().GetCluster().IsBackgroundJobsRunning() }) tc.GetPrimaryServer().Close() - // Stop scheduling server. API server will execute scheduling jobs again. + // Stop scheduling server. PD service will execute scheduling jobs again. testutil.Eventually(re, func() bool { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -246,7 +246,7 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { re.NoError(err) defer tc1.Destroy() tc1.WaitForPrimaryServing(re) - // After scheduling server is started, API server will not execute scheduling jobs. + // After scheduling server is started, PD service will not execute scheduling jobs. testutil.Eventually(re, func() bool { return !suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -259,21 +259,21 @@ func (suite *serverTestSuite) TestSchedulingServiceFallback() { func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { re := suite.Require() - // API server will execute scheduling jobs since there is no scheduling server. + // PD service will execute scheduling jobs since there is no scheduling server. testutil.Eventually(re, func() bool { re.NotNil(suite.pdLeader.GetServer()) re.NotNil(suite.pdLeader.GetServer().GetRaftCluster()) return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) leaderServer := suite.pdLeader.GetServer() - // After Disabling scheduling service fallback, the API server will stop scheduling. + // After Disabling scheduling service fallback, the PD service will stop scheduling. conf := leaderServer.GetMicroServiceConfig().Clone() conf.EnableSchedulingFallback = false leaderServer.SetMicroServiceConfig(*conf) testutil.Eventually(re, func() bool { return !suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) - // Enable scheduling service fallback again, the API server will restart scheduling. + // Enable scheduling service fallback again, the PD service will restart scheduling. conf.EnableSchedulingFallback = true leaderServer.SetMicroServiceConfig(*conf) testutil.Eventually(re, func() bool { @@ -284,7 +284,7 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { re.NoError(err) defer tc.Destroy() tc.WaitForPrimaryServing(re) - // After scheduling server is started, API server will not execute scheduling jobs. + // After scheduling server is started, PD service will not execute scheduling jobs. testutil.Eventually(re, func() bool { return !suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -292,7 +292,7 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { testutil.Eventually(re, func() bool { return tc.GetPrimaryServer().GetCluster().IsBackgroundJobsRunning() }) - // Disable scheduling service fallback and stop scheduling server. API server won't execute scheduling jobs again. + // Disable scheduling service fallback and stop scheduling server. PD service won't execute scheduling jobs again. conf.EnableSchedulingFallback = false leaderServer.SetMicroServiceConfig(*conf) tc.GetPrimaryServer().Close() @@ -310,14 +310,14 @@ func (suite *serverTestSuite) TestSchedulerSync() { tc.WaitForPrimaryServing(re) schedulersController := tc.GetPrimaryServer().GetCluster().GetCoordinator().GetSchedulersController() checkEvictLeaderSchedulerExist(re, schedulersController, false) - // Add a new evict-leader-scheduler through the API server. + // Add a new evict-leader-scheduler through the PD service. api.MustAddScheduler(re, suite.backendEndpoints, types.EvictLeaderScheduler.String(), map[string]any{ "store_id": 1, }) // Check if the evict-leader-scheduler is added. checkEvictLeaderSchedulerExist(re, schedulersController, true) checkEvictLeaderStoreIDs(re, schedulersController, []uint64{1}) - // Add a store_id to the evict-leader-scheduler through the API server. + // Add a store_id to the evict-leader-scheduler through the PD service. err = suite.pdLeader.GetServer().GetRaftCluster().PutMetaStore( &metapb.Store{ Id: 2, @@ -334,18 +334,18 @@ func (suite *serverTestSuite) TestSchedulerSync() { }) checkEvictLeaderSchedulerExist(re, schedulersController, true) checkEvictLeaderStoreIDs(re, schedulersController, []uint64{1, 2}) - // Delete a store_id from the evict-leader-scheduler through the API server. + // Delete a store_id from the evict-leader-scheduler through the PD service. api.MustDeleteScheduler(re, suite.backendEndpoints, fmt.Sprintf("%s-%d", types.EvictLeaderScheduler.String(), 1)) checkEvictLeaderSchedulerExist(re, schedulersController, true) checkEvictLeaderStoreIDs(re, schedulersController, []uint64{2}) - // Add a store_id to the evict-leader-scheduler through the API server by the scheduler handler. + // Add a store_id to the evict-leader-scheduler through the PD service by the scheduler handler. api.MustCallSchedulerConfigAPI(re, http.MethodPost, suite.backendEndpoints, types.EvictLeaderScheduler.String(), []string{"config"}, map[string]any{ "name": types.EvictLeaderScheduler.String(), "store_id": 1, }) checkEvictLeaderSchedulerExist(re, schedulersController, true) checkEvictLeaderStoreIDs(re, schedulersController, []uint64{1, 2}) - // Delete a store_id from the evict-leader-scheduler through the API server by the scheduler handler. + // Delete a store_id from the evict-leader-scheduler through the PD service by the scheduler handler. api.MustCallSchedulerConfigAPI(re, http.MethodDelete, suite.backendEndpoints, types.EvictLeaderScheduler.String(), []string{"delete", "2"}, nil) checkEvictLeaderSchedulerExist(re, schedulersController, true) checkEvictLeaderStoreIDs(re, schedulersController, []uint64{1}) @@ -354,7 +354,7 @@ func (suite *serverTestSuite) TestSchedulerSync() { // Check if the scheduler is removed. checkEvictLeaderSchedulerExist(re, schedulersController, false) - // Delete the evict-leader-scheduler through the API server by removing the last store_id. + // Delete the evict-leader-scheduler through the PD service by removing the last store_id. api.MustAddScheduler(re, suite.backendEndpoints, types.EvictLeaderScheduler.String(), map[string]any{ "store_id": 1, }) @@ -363,7 +363,7 @@ func (suite *serverTestSuite) TestSchedulerSync() { api.MustDeleteScheduler(re, suite.backendEndpoints, fmt.Sprintf("%s-%d", types.EvictLeaderScheduler.String(), 1)) checkEvictLeaderSchedulerExist(re, schedulersController, false) - // Delete the evict-leader-scheduler through the API server. + // Delete the evict-leader-scheduler through the PD service. api.MustAddScheduler(re, suite.backendEndpoints, types.EvictLeaderScheduler.String(), map[string]any{ "store_id": 1, }) @@ -551,7 +551,7 @@ func (suite *serverTestSuite) TestStoreLimit() { leaderServer.GetRaftCluster().SetStoreLimit(1, storelimit.RemovePeer, 60) leaderServer.GetRaftCluster().SetStoreLimit(2, storelimit.AddPeer, 60) leaderServer.GetRaftCluster().SetStoreLimit(2, storelimit.RemovePeer, 60) - // There is a time window between setting store limit in API service side and capturing the change in scheduling service. + // There is a time window between setting store limit in PD service side and capturing the change in scheduling service. waitSyncFinish(re, tc, storelimit.AddPeer, 60) for i := uint64(1); i <= 5; i++ { op := operator.NewTestOperator(2, &metapb.RegionEpoch{}, operator.OpRegion, operator.AddPeer{ToStore: 2, PeerID: 100}) @@ -636,7 +636,7 @@ func (suite *multipleServerTestSuite) SetupSuite() { re := suite.Require() re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs", `return(true)`)) suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 2) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 2) re.NoError(err) err = suite.cluster.RunInitialServers() diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index 91614530ef1..dceb5ccdf7c 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -62,7 +62,7 @@ func (suite *tsoAPITestSuite) SetupTest() { var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.pdCluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.pdCluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.pdCluster.RunInitialServers() re.NoError(err) @@ -137,7 +137,7 @@ func TestTSOServerStartFirst(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - apiCluster, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + apiCluster, err := tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = []string{"k1", "k2"} }) defer apiCluster.Destroy() @@ -200,7 +200,7 @@ func TestForwardOnlyTSONoScheduling(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - tc, err := tests.NewTestAPICluster(ctx, 1) + tc, err := tests.NewTestPDServiceCluster(ctx, 1) defer tc.Destroy() re.NoError(err) err = tc.RunInitialServers() @@ -227,7 +227,7 @@ func TestForwardOnlyTSONoScheduling(t *testing.T) { testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully"), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) re.NoError(err) - // If close tso server, it should try forward to tso server, but return error in api mode. + // If close tso server, it should try forward to tso server, but return error in pd service mode. ttc.Destroy() err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "admin/reset-ts"), input, testutil.Status(re, http.StatusInternalServerError), testutil.StringContain(re, "[PD:apiutil:ErrRedirect]redirect failed")) diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index 2c19f6588e5..ecbc0295845 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -82,7 +82,7 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) SetupSuite() { var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() re.NoError(err) @@ -537,8 +537,8 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) - // Init api server config but not start. - tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + // Init PD service config but not start. + tc, err := tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = []string{ "keyspace_a", "keyspace_b", } @@ -546,7 +546,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - // Start api server and tso server. + // Start PD service and tso server. err = tc.RunInitialServers() re.NoError(err) defer tc.Destroy() @@ -734,8 +734,8 @@ func TestGetTSOImmediately(t *testing.T) { re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) - // Init api server config but not start. - tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + // Init PD service config but not start. + tc, err := tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = []string{ "keyspace_a", "keyspace_b", } @@ -743,7 +743,7 @@ func TestGetTSOImmediately(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - // Start api server and tso server. + // Start PD service and tso server. err = tc.RunInitialServers() re.NoError(err) defer tc.Destroy() diff --git a/tests/integrations/mcs/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index b564076c1f0..50583ebbbb4 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -62,7 +62,7 @@ func (s *tsoProxyTestSuite) SetupSuite() { var err error s.ctx, s.cancel = context.WithCancel(context.Background()) // Create an API cluster with 1 server - s.apiCluster, err = tests.NewTestAPICluster(s.ctx, 1) + s.apiCluster, err = tests.NewTestPDServiceCluster(s.ctx, 1) re.NoError(err) err = s.apiCluster.RunInitialServers() re.NoError(err) diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index 09a199c2d52..7416a314949 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -75,7 +75,7 @@ func (suite *tsoServerTestSuite) SetupSuite() { re := suite.Require() suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() @@ -156,19 +156,19 @@ func (suite *tsoServerTestSuite) TestParticipantStartWithAdvertiseListenAddr() { func TestTSOPath(t *testing.T) { re := require.New(t) - checkTSOPath(re, true /*isAPIServiceMode*/) - checkTSOPath(re, false /*isAPIServiceMode*/) + checkTSOPath(re, true /*isPDServiceMode*/) + checkTSOPath(re, false /*isPDServiceMode*/) } -func checkTSOPath(re *require.Assertions, isAPIServiceMode bool) { +func checkTSOPath(re *require.Assertions, isPDServiceMode bool) { var ( cluster *tests.TestCluster err error ) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - if isAPIServiceMode { - cluster, err = tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + if isPDServiceMode { + cluster, err = tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.MicroService.EnableTSODynamicSwitching = false }) } else { @@ -184,7 +184,7 @@ func checkTSOPath(re *require.Assertions, isAPIServiceMode bool) { re.NoError(pdLeader.BootstrapCluster()) backendEndpoints := pdLeader.GetAddr() client := pdLeader.GetEtcdClient() - if isAPIServiceMode { + if isPDServiceMode { re.Equal(0, getEtcdTimestampKeyNum(re, client)) } else { re.Equal(1, getEtcdTimestampKeyNum(re, client)) @@ -217,7 +217,7 @@ func getEtcdTimestampKeyNum(re *require.Assertions, client *clientv3.Client) int return count } -type APIServerForward struct { +type PDServiceForward struct { re *require.Assertions ctx context.Context cancel context.CancelFunc @@ -227,13 +227,13 @@ type APIServerForward struct { pdClient pd.Client } -func NewAPIServerForward(re *require.Assertions) APIServerForward { - suite := APIServerForward{ +func NewPDServiceForward(re *require.Assertions) PDServiceForward { + suite := PDServiceForward{ re: re, } var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 3) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 3) re.NoError(err) err = suite.cluster.RunInitialServers() @@ -254,7 +254,7 @@ func NewAPIServerForward(re *require.Assertions) APIServerForward { return suite } -func (suite *APIServerForward) ShutDown() { +func (suite *PDServiceForward) ShutDown() { suite.pdClient.Close() re := suite.re @@ -273,7 +273,7 @@ func (suite *APIServerForward) ShutDown() { func TestForwardTSORelated(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() leaderServer := suite.cluster.GetLeaderServer().GetServer() cfg := leaderServer.GetMicroServiceConfig().Clone() @@ -290,7 +290,7 @@ func TestForwardTSORelated(t *testing.T) { func TestForwardTSOWhenPrimaryChanged(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) @@ -330,7 +330,7 @@ func TestForwardTSOWhenPrimaryChanged(t *testing.T) { func TestResignTSOPrimaryForward(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() // TODO: test random kill primary with 3 nodes tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) @@ -356,7 +356,7 @@ func TestResignTSOPrimaryForward(t *testing.T) { func TestResignAPIPrimaryForward(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) @@ -380,7 +380,7 @@ func TestResignAPIPrimaryForward(t *testing.T) { func TestForwardTSOUnexpectedToFollower1(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() suite.checkForwardTSOUnexpectedToFollower(func() { // unary call will retry internally @@ -393,7 +393,7 @@ func TestForwardTSOUnexpectedToFollower1(t *testing.T) { func TestForwardTSOUnexpectedToFollower2(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() suite.checkForwardTSOUnexpectedToFollower(func() { // unary call will retry internally @@ -407,7 +407,7 @@ func TestForwardTSOUnexpectedToFollower2(t *testing.T) { func TestForwardTSOUnexpectedToFollower3(t *testing.T) { re := require.New(t) - suite := NewAPIServerForward(re) + suite := NewPDServiceForward(re) defer suite.ShutDown() suite.checkForwardTSOUnexpectedToFollower(func() { _, _, err := suite.pdClient.GetTS(suite.ctx) @@ -415,7 +415,7 @@ func TestForwardTSOUnexpectedToFollower3(t *testing.T) { }) } -func (suite *APIServerForward) checkForwardTSOUnexpectedToFollower(checkTSO func()) { +func (suite *PDServiceForward) checkForwardTSOUnexpectedToFollower(checkTSO func()) { re := suite.re tc, err := tests.NewTestTSOCluster(suite.ctx, 2, suite.backendEndpoints) re.NoError(err) @@ -451,7 +451,7 @@ func (suite *APIServerForward) checkForwardTSOUnexpectedToFollower(checkTSO func tc.Destroy() } -func (suite *APIServerForward) addRegions() { +func (suite *PDServiceForward) addRegions() { leader := suite.cluster.GetServer(suite.cluster.WaitLeader()) rc := leader.GetServer().GetRaftCluster() for i := range 3 { @@ -465,7 +465,7 @@ func (suite *APIServerForward) addRegions() { } } -func (suite *APIServerForward) checkUnavailableTSO(re *require.Assertions) { +func (suite *PDServiceForward) checkUnavailableTSO(re *require.Assertions) { _, _, err := suite.pdClient.GetTS(suite.ctx) re.Error(err) // try to update gc safe point @@ -476,7 +476,7 @@ func (suite *APIServerForward) checkUnavailableTSO(re *require.Assertions) { re.Error(err) } -func (suite *APIServerForward) checkAvailableTSO(re *require.Assertions) { +func (suite *PDServiceForward) checkAvailableTSO(re *require.Assertions) { mcs.WaitForTSOServiceAvailable(suite.ctx, re, suite.pdClient) // try to get ts _, _, err := suite.pdClient.GetTS(suite.ctx) @@ -512,7 +512,7 @@ func (suite *CommonTestSuite) SetupSuite() { var err error re := suite.Require() suite.ctx, suite.cancel = context.WithCancel(context.Background()) - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, 1) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) err = suite.cluster.RunInitialServers() @@ -576,7 +576,7 @@ func (suite *CommonTestSuite) TestBootstrapDefaultKeyspaceGroup() { } check() - s, err := suite.cluster.JoinAPIServer(suite.ctx) + s, err := suite.cluster.JoinPDServer(suite.ctx) re.NoError(err) re.NoError(s.Run()) @@ -598,7 +598,7 @@ func TestTSOServiceSwitch(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - tc, err := tests.NewTestAPICluster(ctx, 1, + tc, err := tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.MicroService.EnableTSODynamicSwitching = true }, diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index 2cda9f8734f..a06e44ed4ab 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -98,7 +98,7 @@ func (suite *tsoClientTestSuite) SetupSuite() { if suite.legacy { suite.cluster, err = tests.NewTestCluster(suite.ctx, serverCount) } else { - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, serverCount, func(conf *config.Config, _ string) { + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, serverCount, func(conf *config.Config, _ string) { conf.MicroService.EnableTSODynamicSwitching = false }) } @@ -510,7 +510,7 @@ func TestMixedTSODeployment(t *testing.T) { re.NotNil(leaderServer) backendEndpoints := leaderServer.GetAddr() - apiSvr, err := cluster.JoinAPIServer(ctx) + apiSvr, err := cluster.JoinPDServer(ctx) re.NoError(err) err = apiSvr.Run() re.NoError(err) @@ -544,7 +544,7 @@ func TestUpgradingAPIandTSOClusters(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) // Create an API cluster which has 3 servers - apiCluster, err := tests.NewTestAPICluster(ctx, 3) + apiCluster, err := tests.NewTestPDServiceCluster(ctx, 3) re.NoError(err) err = apiCluster.RunInitialServers() re.NoError(err) diff --git a/tests/integrations/tso/consistency_test.go b/tests/integrations/tso/consistency_test.go index 147f41a4591..b29ae696f26 100644 --- a/tests/integrations/tso/consistency_test.go +++ b/tests/integrations/tso/consistency_test.go @@ -76,7 +76,7 @@ func (suite *tsoConsistencyTestSuite) SetupSuite() { if suite.legacy { suite.cluster, err = tests.NewTestCluster(suite.ctx, serverCount) } else { - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, serverCount) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, serverCount) } re.NoError(err) err = suite.cluster.RunInitialServers() diff --git a/tests/integrations/tso/server_test.go b/tests/integrations/tso/server_test.go index f03db197b35..1428dbcd1a6 100644 --- a/tests/integrations/tso/server_test.go +++ b/tests/integrations/tso/server_test.go @@ -74,7 +74,7 @@ func (suite *tsoServerTestSuite) SetupSuite() { if suite.legacy { suite.cluster, err = tests.NewTestCluster(suite.ctx, serverCount) } else { - suite.cluster, err = tests.NewTestAPICluster(suite.ctx, serverCount) + suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, serverCount) } re.NoError(err) err = suite.cluster.RunInitialServers() diff --git a/tests/server/api/scheduler_test.go b/tests/server/api/scheduler_test.go index 1f76c469cfd..d1d5b06ceb4 100644 --- a/tests/server/api/scheduler_test.go +++ b/tests/server/api/scheduler_test.go @@ -55,7 +55,7 @@ func TestPDSchedulingTestSuite(t *testing.T) { func TestAPISchedulingTestSuite(t *testing.T) { suite.Run(t, &scheduleTestSuite{ - runMode: tests.APIMode, + runMode: tests.PDServiceMode, }) } diff --git a/tests/server/apiv2/handlers/tso_keyspace_group_test.go b/tests/server/apiv2/handlers/tso_keyspace_group_test.go index 796fd514eef..851df9b5fd1 100644 --- a/tests/server/apiv2/handlers/tso_keyspace_group_test.go +++ b/tests/server/apiv2/handlers/tso_keyspace_group_test.go @@ -42,7 +42,7 @@ func TestKeyspaceGroupTestSuite(t *testing.T) { func (suite *keyspaceGroupTestSuite) SetupTest() { re := suite.Require() suite.ctx, suite.cancel = context.WithCancel(context.Background()) - cluster, err := tests.NewTestAPICluster(suite.ctx, 1) + cluster, err := tests.NewTestPDServiceCluster(suite.ctx, 1) suite.cluster = cluster re.NoError(err) re.NoError(cluster.RunInitialServers()) diff --git a/tests/server/server_test.go b/tests/server/server_test.go index 77cd7aa5158..06623b6f092 100644 --- a/tests/server/server_test.go +++ b/tests/server/server_test.go @@ -67,7 +67,7 @@ func TestUpdateAdvertiseUrls(t *testing.T) { for _, conf := range cluster.GetConfig().InitialServers { serverConf, err := conf.Generate() re.NoError(err) - s, err := tests.NewTestServer(ctx, serverConf) + s, err := tests.NewTestServer(ctx, serverConf, nil) re.NoError(err) cluster.GetServers()[conf.Name] = s } diff --git a/tests/testutil.go b/tests/testutil.go index 5e99b3dbeda..4bbfa8155b4 100644 --- a/tests/testutil.go +++ b/tests/testutil.go @@ -279,8 +279,8 @@ const ( Both SchedulerMode = iota // PDMode represents PD mode. PDMode - // APIMode represents API mode. - APIMode + // PDServiceMode represents API mode. + PDServiceMode ) // SchedulingTestEnvironment is used for test purpose. @@ -308,11 +308,11 @@ func (s *SchedulingTestEnvironment) RunTestBasedOnMode(test func(*TestCluster)) switch s.RunMode { case PDMode: s.RunTestInPDMode(test) - case APIMode: - s.RunTestInAPIMode(test) + case PDServiceMode: + s.RunTestInPDServiceMode(test) default: s.RunTestInPDMode(test) - s.RunTestInAPIMode(test) + s.RunTestInPDServiceMode(test) } } @@ -339,8 +339,8 @@ func getTestName() string { return "" } -// RunTestInAPIMode is to run test in api mode. -func (s *SchedulingTestEnvironment) RunTestInAPIMode(test func(*TestCluster)) { +// RunTestInPDServiceMode is to run test in pd service mode. +func (s *SchedulingTestEnvironment) RunTestInPDServiceMode(test func(*TestCluster)) { re := require.New(s.t) re.NoError(failpoint.Enable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs", `return(true)`)) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/mcs/scheduling/server/fastUpdateMember", `return(true)`)) @@ -348,11 +348,11 @@ func (s *SchedulingTestEnvironment) RunTestInAPIMode(test func(*TestCluster)) { re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/mcs/scheduling/server/fastUpdateMember")) re.NoError(failpoint.Disable("github.com/tikv/pd/server/cluster/highFrequencyClusterJobs")) }() - s.t.Logf("start test %s in api mode", getTestName()) - if _, ok := s.clusters[APIMode]; !ok { - s.startCluster(APIMode) + s.t.Logf("start test %s in pd service mode", getTestName()) + if _, ok := s.clusters[PDServiceMode]; !ok { + s.startCluster(PDServiceMode) } - test(s.clusters[APIMode]) + test(s.clusters[PDServiceMode]) } // Cleanup is to cleanup the environment. @@ -379,8 +379,8 @@ func (s *SchedulingTestEnvironment) startCluster(m SchedulerMode) { leaderServer := cluster.GetServer(cluster.GetLeader()) re.NoError(leaderServer.BootstrapCluster()) s.clusters[PDMode] = cluster - case APIMode: - cluster, err := NewTestAPICluster(ctx, 1, s.opts...) + case PDServiceMode: + cluster, err := NewTestPDServiceCluster(ctx, 1, s.opts...) re.NoError(err) err = cluster.RunInitialServers() re.NoError(err) @@ -398,7 +398,7 @@ func (s *SchedulingTestEnvironment) startCluster(m SchedulerMode) { testutil.Eventually(re, func() bool { return cluster.GetLeaderServer().GetServer().IsServiceIndependent(constant.SchedulingServiceName) }) - s.clusters[APIMode] = cluster + s.clusters[PDServiceMode] = cluster } } diff --git a/tools/pd-ctl/pdctl/command/config_command.go b/tools/pd-ctl/pdctl/command/config_command.go index 2e9903db550..f89c63bc51c 100644 --- a/tools/pd-ctl/pdctl/command/config_command.go +++ b/tools/pd-ctl/pdctl/command/config_command.go @@ -49,8 +49,8 @@ const ( 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" + // flagFromPDService has no influence for pd mode, but it is useful for us to debug in pd service mode. + flagFromPDService = "from_pd_service" ) // NewConfigCommand return a config subcommand of rootCmd @@ -81,7 +81,7 @@ func NewShowConfigCommand() *cobra.Command { sc.AddCommand(newShowReplicationModeCommand()) sc.AddCommand(NewShowServerConfigCommand()) sc.AddCommand(NewShowServiceMiddlewareConfigCommand()) - sc.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") return sc } @@ -92,7 +92,7 @@ func NewShowAllConfigCommand() *cobra.Command { Short: "show all config of PD", Run: showAllConfigCommandFunc, } - sc.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") return sc } @@ -103,7 +103,7 @@ func NewShowScheduleConfigCommand() *cobra.Command { Short: "show schedule config of PD", Run: showScheduleConfigCommandFunc, } - sc.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") return sc } @@ -114,7 +114,7 @@ func NewShowReplicationConfigCommand() *cobra.Command { Short: "show replication config of PD", Run: showReplicationConfigCommandFunc, } - sc.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") return sc } @@ -528,7 +528,7 @@ func NewPlacementRulesCommand() *cobra.Command { show.Flags().String("id", "", "rule id") show.Flags().String("region", "", "region id") show.Flags().Bool("detail", false, "detailed match info for region") - show.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + show.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") load := &cobra.Command{ Use: "load", Short: "load placement rules to a file", @@ -538,7 +538,7 @@ func NewPlacementRulesCommand() *cobra.Command { load.Flags().String("id", "", "rule id") load.Flags().String("region", "", "region id") load.Flags().String("out", "rules.json", "the filename contains rules") - load.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + load.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") save := &cobra.Command{ Use: "save", Short: "save rules from file", @@ -554,7 +554,7 @@ func NewPlacementRulesCommand() *cobra.Command { Short: "show rule group configuration(s)", Run: showRuleGroupFunc, } - ruleGroupShow.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + ruleGroupShow.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") ruleGroupSet := &cobra.Command{ Use: "set ", Short: "update rule group configuration", @@ -577,7 +577,7 @@ func NewPlacementRulesCommand() *cobra.Command { Run: getRuleBundle, } ruleBundleGet.Flags().String("out", "", "the output file") - ruleBundleGet.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + ruleBundleGet.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") ruleBundleSet := &cobra.Command{ Use: "set", Short: "set rule group config and its rules from file", @@ -596,7 +596,7 @@ func NewPlacementRulesCommand() *cobra.Command { Run: loadRuleBundle, } ruleBundleLoad.Flags().String("out", "rules.json", "the output file") - ruleBundleLoad.Flags().Bool(flagFromAPIServer, false, "read data from api server rather than micro service") + ruleBundleLoad.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") ruleBundleSave := &cobra.Command{ Use: "save", Short: "save all group configs and rules from file", @@ -895,7 +895,7 @@ func saveRuleBundle(cmd *cobra.Command, _ []string) { func buildHeader(cmd *cobra.Command) http.Header { header := http.Header{} - forbiddenRedirectToMicroService, err := cmd.Flags().GetBool(flagFromAPIServer) + forbiddenRedirectToMicroService, err := cmd.Flags().GetBool(flagFromPDService) if err == nil && forbiddenRedirectToMicroService { header.Add(apiutil.XForbiddenForwardToMicroServiceHeader, "true") } diff --git a/tools/pd-ctl/tests/config/config_test.go b/tools/pd-ctl/tests/config/config_test.go index b6c58fe2bc6..39820a6c7b7 100644 --- a/tools/pd-ctl/tests/config/config_test.go +++ b/tools/pd-ctl/tests/config/config_test.go @@ -383,9 +383,9 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu f.Close() defer os.RemoveAll(fname) - checkScheduleConfig := func(scheduleCfg *sc.ScheduleConfig, isFromAPIServer bool) { + checkScheduleConfig := func(scheduleCfg *sc.ScheduleConfig, isFromPDService bool) { if schedulingServer := cluster.GetSchedulingPrimaryServer(); schedulingServer != nil { - if isFromAPIServer { + if isFromPDService { re.Equal(scheduleCfg.LeaderScheduleLimit, leaderServer.GetPersistOptions().GetLeaderScheduleLimit()) re.NotEqual(scheduleCfg.LeaderScheduleLimit, schedulingServer.GetPersistConfig().GetLeaderScheduleLimit()) } else { @@ -397,9 +397,9 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu } } - checkReplicateConfig := func(replicationCfg *sc.ReplicationConfig, isFromAPIServer bool) { + checkReplicateConfig := func(replicationCfg *sc.ReplicationConfig, isFromPDService bool) { if schedulingServer := cluster.GetSchedulingPrimaryServer(); schedulingServer != nil { - if isFromAPIServer { + if isFromPDService { re.Equal(replicationCfg.MaxReplicas, uint64(leaderServer.GetPersistOptions().GetMaxReplicas())) re.NotEqual(int(replicationCfg.MaxReplicas), schedulingServer.GetPersistConfig().GetMaxReplicas()) } else { @@ -411,11 +411,11 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu } } - checkRules := func(rules []*placement.Rule, isFromAPIServer bool) { + checkRules := func(rules []*placement.Rule, isFromPDService bool) { apiRules := leaderServer.GetRaftCluster().GetRuleManager().GetAllRules() if schedulingServer := cluster.GetSchedulingPrimaryServer(); schedulingServer != nil { schedulingRules := schedulingServer.GetCluster().GetRuleManager().GetAllRules() - if isFromAPIServer { + if isFromPDService { re.Len(apiRules, len(rules)) re.NotEqual(len(schedulingRules), len(rules)) } else { @@ -427,11 +427,11 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu } } - checkGroup := func(group placement.RuleGroup, isFromAPIServer bool) { + checkGroup := func(group placement.RuleGroup, isFromPDService bool) { apiGroup := leaderServer.GetRaftCluster().GetRuleManager().GetRuleGroup(placement.DefaultGroupID) if schedulingServer := cluster.GetSchedulingPrimaryServer(); schedulingServer != nil { schedulingGroup := schedulingServer.GetCluster().GetRuleManager().GetRuleGroup(placement.DefaultGroupID) - if isFromAPIServer { + if isFromPDService { re.Equal(apiGroup.Index, group.Index) re.NotEqual(schedulingGroup.Index, group.Index) } else { @@ -444,28 +444,28 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu } testConfig := func(options ...string) { - for _, isFromAPIServer := range []bool{true, false} { + for _, isFromPDService := range []bool{true, false} { cmd := ctl.GetRootCmd() args := []string{"-u", pdAddr, "config", "show"} args = append(args, options...) - if isFromAPIServer { - args = append(args, "--from_api_server") + if isFromPDService { + args = append(args, "--from_pd_service") } output, err := tests.ExecuteCommand(cmd, args...) re.NoError(err) if len(options) == 0 || options[0] == "all" { cfg := config.Config{} re.NoError(json.Unmarshal(output, &cfg)) - checkReplicateConfig(&cfg.Replication, isFromAPIServer) - checkScheduleConfig(&cfg.Schedule, isFromAPIServer) + checkReplicateConfig(&cfg.Replication, isFromPDService) + checkScheduleConfig(&cfg.Schedule, isFromPDService) } else if options[0] == "replication" { replicationCfg := &sc.ReplicationConfig{} re.NoError(json.Unmarshal(output, replicationCfg)) - checkReplicateConfig(replicationCfg, isFromAPIServer) + checkReplicateConfig(replicationCfg, isFromPDService) } else if options[0] == "schedule" { scheduleCfg := &sc.ScheduleConfig{} re.NoError(json.Unmarshal(output, scheduleCfg)) - checkScheduleConfig(scheduleCfg, isFromAPIServer) + checkScheduleConfig(scheduleCfg, isFromPDService) } else { re.Fail("no implement") } @@ -473,37 +473,37 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu } testRules := func(options ...string) { - for _, isFromAPIServer := range []bool{true, false} { + for _, isFromPDService := range []bool{true, false} { cmd := ctl.GetRootCmd() args := []string{"-u", pdAddr, "config", "placement-rules"} args = append(args, options...) - if isFromAPIServer { - args = append(args, "--from_api_server") + if isFromPDService { + args = append(args, "--from_pd_service") } output, err := tests.ExecuteCommand(cmd, args...) re.NoError(err) if options[0] == "show" { var rules []*placement.Rule re.NoError(json.Unmarshal(output, &rules)) - checkRules(rules, isFromAPIServer) + checkRules(rules, isFromPDService) } else if options[0] == "load" { var rules []*placement.Rule b, _ := os.ReadFile(fname) re.NoError(json.Unmarshal(b, &rules)) - checkRules(rules, isFromAPIServer) + checkRules(rules, isFromPDService) } else if options[0] == "rule-group" { var group placement.RuleGroup re.NoError(json.Unmarshal(output, &group), string(output)) - checkGroup(group, isFromAPIServer) + checkGroup(group, isFromPDService) } else if options[0] == "rule-bundle" && options[1] == "get" { var bundle placement.GroupBundle re.NoError(json.Unmarshal(output, &bundle), string(output)) - checkRules(bundle.Rules, isFromAPIServer) + checkRules(bundle.Rules, isFromPDService) } else if options[0] == "rule-bundle" && options[1] == "load" { var bundles []placement.GroupBundle b, _ := os.ReadFile(fname) re.NoError(json.Unmarshal(b, &bundles), string(output)) - checkRules(bundles[0].Rules, isFromAPIServer) + checkRules(bundles[0].Rules, isFromPDService) } else { re.Fail("no implement") } @@ -522,13 +522,13 @@ func (suite *configTestSuite) checkConfigForwardControl(cluster *pdTests.TestClu re.Equal(uint64(233), sche.GetPersistConfig().GetLeaderScheduleLimit()) re.Equal(7, sche.GetPersistConfig().GetMaxReplicas()) } - // show config from api server rather than scheduling server + // show config from PD service rather than scheduling server testConfig() - // show all config from api server rather than scheduling server + // show all config from PD service rather than scheduling server testConfig("all") - // show replication config from api server rather than scheduling server + // show replication config from PD service rather than scheduling server testConfig("replication") - // show schedule config from api server rather than scheduling server + // show schedule config from PD service rather than scheduling server testConfig("schedule") // Test Rule diff --git a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go index fca00f2fd3c..fff95856931 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_group_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_group_test.go @@ -41,7 +41,7 @@ func TestKeyspaceGroup(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - tc, err := pdTests.NewTestAPICluster(ctx, 1) + tc, err := pdTests.NewTestPDServiceCluster(ctx, 1) re.NoError(err) defer tc.Destroy() err = tc.RunInitialServers() @@ -102,7 +102,7 @@ func TestSplitKeyspaceGroup(t *testing.T) { for i := range 129 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 3, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 3, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -157,7 +157,7 @@ func TestExternalAllocNodeWhenStart(t *testing.T) { for i := range 10 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -197,7 +197,7 @@ func TestSetNodeAndPriorityKeyspaceGroup(t *testing.T) { for i := range 10 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 3, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 3, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -301,7 +301,7 @@ func TestMergeKeyspaceGroup(t *testing.T) { for i := range 129 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -420,7 +420,7 @@ func TestKeyspaceGroupState(t *testing.T) { for i := range 10 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -511,7 +511,7 @@ func TestShowKeyspaceGroupPrimary(t *testing.T) { for i := range 10 { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) diff --git a/tools/pd-ctl/tests/keyspace/keyspace_test.go b/tools/pd-ctl/tests/keyspace/keyspace_test.go index 23a1148cd66..6a523ced7b8 100644 --- a/tools/pd-ctl/tests/keyspace/keyspace_test.go +++ b/tools/pd-ctl/tests/keyspace/keyspace_test.go @@ -49,7 +49,7 @@ func TestKeyspace(t *testing.T) { for i := 1; i < 10; i++ { keyspaces = append(keyspaces, fmt.Sprintf("keyspace_%d", i)) } - tc, err := pdTests.NewTestAPICluster(ctx, 3, func(conf *config.Config, _ string) { + tc, err := pdTests.NewTestPDServiceCluster(ctx, 3, func(conf *config.Config, _ string) { conf.Keyspace.PreAlloc = keyspaces }) re.NoError(err) @@ -155,7 +155,7 @@ func (suite *keyspaceTestSuite) SetupTest() { suite.ctx, suite.cancel = context.WithCancel(context.Background()) re.NoError(failpoint.Enable("github.com/tikv/pd/server/delayStartServerLoop", `return(true)`)) re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/skipSplitRegion", "return(true)")) - tc, err := pdTests.NewTestAPICluster(suite.ctx, 1) + tc, err := pdTests.NewTestPDServiceCluster(suite.ctx, 1) re.NoError(err) re.NoError(tc.RunInitialServers()) tc.WaitLeader() diff --git a/tools/pd-simulator/simulator/drive.go b/tools/pd-simulator/simulator/drive.go index 0d81a2af1ab..d8a6094760e 100644 --- a/tools/pd-simulator/simulator/drive.go +++ b/tools/pd-simulator/simulator/drive.go @@ -165,7 +165,7 @@ func (d *Driver) allocID() error { func (d *Driver) updateNodesClient() error { urls := strings.Split(d.pdAddr, ",") ctx, cancel := context.WithCancel(context.Background()) - SD = sd.NewDefaultPDServiceDiscovery(ctx, cancel, urls, nil) + SD = sd.NewDefaultServiceDiscovery(ctx, cancel, urls, nil) if err := SD.Init(); err != nil { return err } From adddd4eaa62dcff0c0310b36a8d3687894a4bd5a Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 8 Jan 2025 20:12:37 +0800 Subject: [PATCH 33/33] *: fix typo (#8982) ref tikv/pd#4820 Signed-off-by: Ryan Leung --- .github/workflows/pd-tests.yaml | 4 +- client/http/api.go | 16 +-- client/http/interface.go | 24 ++-- client/http/request_info.go | 4 +- client/http/types.go | 4 +- pkg/errs/errno.go | 2 +- pkg/member/election_leader.go | 2 +- pkg/schedule/placement/rule_manager.go | 2 +- pkg/utils/apiutil/apiutil.go | 8 +- pkg/utils/apiutil/serverapi/middleware.go | 16 +-- pkg/utils/keypath/absolute_key_path.go | 2 +- server/api/config.go | 14 +-- server/apiv2/handlers/micro_service.go | 8 +- server/apiv2/router.go | 2 +- server/cluster/cluster.go | 4 +- server/config/config.go | 18 +-- server/config/persist_options.go | 20 ++-- server/server.go | 28 ++--- tests/integrations/mcs/members/member_test.go | 36 +++--- tests/integrations/mcs/scheduling/api_test.go | 112 +++++++++--------- .../mcs/scheduling/server_test.go | 12 +- tests/integrations/mcs/tso/api_test.go | 8 +- tests/integrations/mcs/tso/server_test.go | 16 +-- tests/integrations/tso/client_test.go | 2 +- tools/pd-ctl/pdctl/command/config_command.go | 24 ++-- tools/pd-ctl/pdctl/command/global.go | 6 +- tools/pd-ctl/tests/config/config_test.go | 12 +- 27 files changed, 203 insertions(+), 203 deletions(-) diff --git a/.github/workflows/pd-tests.yaml b/.github/workflows/pd-tests.yaml index 1464c696db3..2379f67d759 100644 --- a/.github/workflows/pd-tests.yaml +++ b/.github/workflows/pd-tests.yaml @@ -43,9 +43,9 @@ jobs: - worker_id: 8 name: 'TSO Integration Test' - worker_id: 9 - name: 'MicroService Integration(!TSO)' + name: 'Microservice Integration(!TSO)' - worker_id: 10 - name: 'MicroService Integration(TSO)' + name: 'Microservice Integration(TSO)' outputs: job-total: 10 steps: diff --git a/client/http/api.go b/client/http/api.go index 42d965ec17c..028af858904 100644 --- a/client/http/api.go +++ b/client/http/api.go @@ -80,8 +80,8 @@ const ( Version = "/pd/api/v1/version" operators = "/pd/api/v1/operators" safepoint = "/pd/api/v1/gc/safepoint" - // Micro Service - microServicePrefix = "/pd/api/v2/ms" + // Microservice + microservicePrefix = "/pd/api/v2/ms" // Keyspace KeyspaceConfig = "/pd/api/v2/keyspaces/%s/config" GetKeyspaceMetaByName = "/pd/api/v2/keyspaces/%s" @@ -198,14 +198,14 @@ func PProfGoroutineWithDebugLevel(level int) string { return fmt.Sprintf("%s?debug=%d", PProfGoroutine, level) } -// MicroServiceMembers returns the path of PD HTTP API to get the members of microservice. -func MicroServiceMembers(service string) string { - return fmt.Sprintf("%s/members/%s", microServicePrefix, service) +// MicroserviceMembers returns the path of PD HTTP API to get the members of microservice. +func MicroserviceMembers(service string) string { + return fmt.Sprintf("%s/members/%s", microservicePrefix, service) } -// MicroServicePrimary returns the path of PD HTTP API to get the primary of microservice. -func MicroServicePrimary(service string) string { - return fmt.Sprintf("%s/primary/%s", microServicePrefix, service) +// MicroservicePrimary returns the path of PD HTTP API to get the primary of microservice. +func MicroservicePrimary(service string) string { + return fmt.Sprintf("%s/primary/%s", microservicePrefix, service) } // GetUpdateKeyspaceConfigURL returns the path of PD HTTP API to update keyspace config. diff --git a/client/http/interface.go b/client/http/interface.go index 1aabd1ae331..3008824edb8 100644 --- a/client/http/interface.go +++ b/client/http/interface.go @@ -103,9 +103,9 @@ type Client interface { GetPDVersion(context.Context) (string, error) GetGCSafePoint(context.Context) (ListServiceGCSafepoint, error) DeleteGCSafePoint(context.Context, string) (string, error) - /* Micro Service interfaces */ - GetMicroServiceMembers(context.Context, string) ([]MicroServiceMember, error) - GetMicroServicePrimary(context.Context, string) (string, error) + /* Microservice interfaces */ + GetMicroserviceMembers(context.Context, string) ([]MicroserviceMember, error) + GetMicroservicePrimary(context.Context, string) (string, error) DeleteOperators(context.Context) error /* Keyspace interface */ @@ -937,12 +937,12 @@ func (c *client) GetMinResolvedTSByStoresIDs(ctx context.Context, storeIDs []uin return resp.MinResolvedTS, resp.StoresMinResolvedTS, nil } -// GetMicroServiceMembers gets the members of the microservice. -func (c *client) GetMicroServiceMembers(ctx context.Context, service string) ([]MicroServiceMember, error) { - var members []MicroServiceMember +// GetMicroserviceMembers gets the members of the microservice. +func (c *client) GetMicroserviceMembers(ctx context.Context, service string) ([]MicroserviceMember, error) { + var members []MicroserviceMember err := c.request(ctx, newRequestInfo(). - WithName(getMicroServiceMembersName). - WithURI(MicroServiceMembers(service)). + WithName(getMicroserviceMembersName). + WithURI(MicroserviceMembers(service)). WithMethod(http.MethodGet). WithResp(&members)) if err != nil { @@ -951,12 +951,12 @@ func (c *client) GetMicroServiceMembers(ctx context.Context, service string) ([] return members, nil } -// GetMicroServicePrimary gets the primary of the microservice. -func (c *client) GetMicroServicePrimary(ctx context.Context, service string) (string, error) { +// GetMicroservicePrimary gets the primary of the microservice. +func (c *client) GetMicroservicePrimary(ctx context.Context, service string) (string, error) { var primary string err := c.request(ctx, newRequestInfo(). - WithName(getMicroServicePrimaryName). - WithURI(MicroServicePrimary(service)). + WithName(getMicroservicePrimaryName). + WithURI(MicroservicePrimary(service)). WithMethod(http.MethodGet). WithResp(&primary)) return primary, err diff --git a/client/http/request_info.go b/client/http/request_info.go index d1930800304..2cc95e0215a 100644 --- a/client/http/request_info.go +++ b/client/http/request_info.go @@ -76,8 +76,8 @@ const ( accelerateScheduleName = "AccelerateSchedule" accelerateScheduleInBatchName = "AccelerateScheduleInBatch" getMinResolvedTSByStoresIDsName = "GetMinResolvedTSByStoresIDs" - getMicroServiceMembersName = "GetMicroServiceMembers" - getMicroServicePrimaryName = "GetMicroServicePrimary" + getMicroserviceMembersName = "GetMicroserviceMembers" + getMicroservicePrimaryName = "GetMicroservicePrimary" getPDVersionName = "GetPDVersion" resetTSName = "ResetTS" resetBaseAllocIDName = "ResetBaseAllocID" diff --git a/client/http/types.go b/client/http/types.go index 83e8badf334..75e9db127ff 100644 --- a/client/http/types.go +++ b/client/http/types.go @@ -605,8 +605,8 @@ type MembersInfo struct { EtcdLeader *pdpb.Member `json:"etcd_leader,omitempty"` } -// MicroServiceMember is the member info of a micro service. -type MicroServiceMember struct { +// MicroserviceMember is the member info of a microservice. +type MicroserviceMember struct { ServiceAddr string `json:"service-addr"` Version string `json:"version"` GitHash string `json:"git-hash"` diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 834bf4f824e..25f69af327f 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -542,7 +542,7 @@ var ( ErrInvalidGroup = errors.Normalize("invalid group settings, please check the group name, priority and the number of resources", errors.RFCCodeText("PD:resourcemanager:ErrInvalidGroup")) ) -// Micro service errors +// Microservice errors var ( ErrNotFoundSchedulingPrimary = errors.Normalize("cannot find scheduling primary", errors.RFCCodeText("PD:mcs:ErrNotFoundSchedulingPrimary")) ErrSchedulingServer = errors.Normalize("scheduling server meets %v", errors.RFCCodeText("PD:mcs:ErrSchedulingServer")) diff --git a/pkg/member/election_leader.go b/pkg/member/election_leader.go index 2e5769d7dc4..5cdc7b4cd9b 100644 --- a/pkg/member/election_leader.go +++ b/pkg/member/election_leader.go @@ -21,7 +21,7 @@ import ( ) // ElectionLeader defines the common interface of the leader, which is the pdpb.Member -// for in PD/PD service or the tsopb.Participant in the micro services. +// for in PD/PD service or the tsopb.Participant in the microservices. type ElectionLeader interface { // GetListenUrls returns the listen urls GetListenUrls() []string diff --git a/pkg/schedule/placement/rule_manager.go b/pkg/schedule/placement/rule_manager.go index 4470ff28424..7dc24de0f60 100644 --- a/pkg/schedule/placement/rule_manager.go +++ b/pkg/schedule/placement/rule_manager.go @@ -88,7 +88,7 @@ func (m *RuleManager) Initialize(maxReplica int, locationLabels []string, isolat if m.initialized { return nil } - // If RuleManager is initialized in micro service, + // If RuleManager is initialized in microservice, // it will load from etcd watcher and do not modify rule directly. if skipLoadRules { m.ruleList = ruleList{ diff --git a/pkg/utils/apiutil/apiutil.go b/pkg/utils/apiutil/apiutil.go index a743c543468..0b064b6d91a 100644 --- a/pkg/utils/apiutil/apiutil.go +++ b/pkg/utils/apiutil/apiutil.go @@ -64,10 +64,10 @@ const ( XRealIPHeader = "X-Real-Ip" // XCallerIDHeader is used to mark the caller ID. XCallerIDHeader = "X-Caller-ID" - // XForbiddenForwardToMicroServiceHeader is used to indicate that forwarding the request to a microservice is explicitly disallowed. - XForbiddenForwardToMicroServiceHeader = "X-Forbidden-Forward-To-MicroService" - // XForwardedToMicroServiceHeader is used to signal that the request has already been forwarded to a microservice. - XForwardedToMicroServiceHeader = "X-Forwarded-To-MicroService" + // XForbiddenForwardToMicroserviceHeader is used to indicate that forwarding the request to a microservice is explicitly disallowed. + XForbiddenForwardToMicroserviceHeader = "X-Forbidden-Forward-To-Microservice" + // XForwardedToMicroserviceHeader is used to signal that the request has already been forwarded to a microservice. + XForwardedToMicroserviceHeader = "X-Forwarded-To-Microservice" chunkSize = 4096 ) diff --git a/pkg/utils/apiutil/serverapi/middleware.go b/pkg/utils/apiutil/serverapi/middleware.go index 823deed64ea..0b03c787e6b 100644 --- a/pkg/utils/apiutil/serverapi/middleware.go +++ b/pkg/utils/apiutil/serverapi/middleware.go @@ -115,14 +115,14 @@ func MicroserviceRedirectRule(matchPath, targetPath, targetServiceName string, } } -func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, string) { +func (h *redirector) matchMicroserviceRedirectRules(r *http.Request) (bool, string) { if !h.s.IsPDServiceMode() { return false, "" } if len(h.microserviceRedirectRules) == 0 { return false, "" } - if r.Header.Get(apiutil.XForbiddenForwardToMicroServiceHeader) == "true" { + if r.Header.Get(apiutil.XForbiddenForwardToMicroserviceHeader) == "true" { return false, "" } // Remove trailing '/' from the URL path @@ -166,7 +166,7 @@ func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, stri } else { r.URL.Path = rule.targetPath } - log.Debug("redirect to micro service", zap.String("path", r.URL.Path), zap.String("origin-path", origin), + log.Debug("redirect to microservice", zap.String("path", r.URL.Path), zap.String("origin-path", origin), zap.String("target", addr), zap.String("method", r.Method)) return true, addr } @@ -175,7 +175,7 @@ func (h *redirector) matchMicroServiceRedirectRules(r *http.Request) (bool, stri } func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http.HandlerFunc) { - redirectToMicroService, targetAddr := h.matchMicroServiceRedirectRules(r) + redirectToMicroservice, targetAddr := h.matchMicroserviceRedirectRules(r) allowFollowerHandle := len(r.Header.Get(apiutil.PDAllowFollowerHandleHeader)) > 0 if h.s.IsClosed() { @@ -183,7 +183,7 @@ func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http return } - if (allowFollowerHandle || h.s.GetMember().IsLeader()) && !redirectToMicroService { + if (allowFollowerHandle || h.s.GetMember().IsLeader()) && !redirectToMicroservice { next(w, r) return } @@ -200,14 +200,14 @@ func (h *redirector) ServeHTTP(w http.ResponseWriter, r *http.Request, next http } var clientUrls []string - if redirectToMicroService { + if redirectToMicroservice { if len(targetAddr) == 0 { http.Error(w, errs.ErrRedirect.FastGenByArgs().Error(), http.StatusInternalServerError) return } clientUrls = append(clientUrls, targetAddr) - // Add a header to the response, it is used to mark whether the request has been forwarded to the micro service. - w.Header().Add(apiutil.XForwardedToMicroServiceHeader, "true") + // Add a header to the response, it is used to mark whether the request has been forwarded to the microservice. + w.Header().Add(apiutil.XForwardedToMicroserviceHeader, "true") } else if name := r.Header.Get(apiutil.PDRedirectorHeader); len(name) == 0 { leader := h.waitForLeader(r) // The leader has not been elected yet. diff --git a/pkg/utils/keypath/absolute_key_path.go b/pkg/utils/keypath/absolute_key_path.go index 40692dc14bd..c3acbe6ea9c 100644 --- a/pkg/utils/keypath/absolute_key_path.go +++ b/pkg/utils/keypath/absolute_key_path.go @@ -44,7 +44,7 @@ const ( msTsoKespaceExpectedLeaderPathFormat = "/ms/%d/tso/keyspace_groups/election/%05d/primary/expected_primary" // "/ms/{cluster_id}/tso/keyspace_groups/election/{group_id}/primary" ) -// MsParam is the parameter of micro service. +// MsParam is the parameter of microservice. type MsParam struct { ServiceName string GroupID uint32 // only used for tso keyspace group diff --git a/server/api/config.go b/server/api/config.go index a27a1ed5e9b..ff4fe0add82 100644 --- a/server/api/config.go +++ b/server/api/config.go @@ -65,7 +65,7 @@ func newConfHandler(svr *server.Server, rd *render.Render) *confHandler { func (h *confHandler) GetConfig(w http.ResponseWriter, r *http.Request) { cfg := h.svr.GetConfig() if h.svr.IsServiceIndependent(constant.SchedulingServiceName) && - r.Header.Get(apiutil.XForbiddenForwardToMicroServiceHeader) != "true" { + r.Header.Get(apiutil.XForbiddenForwardToMicroserviceHeader) != "true" { schedulingServerConfig, err := h.getSchedulingServerConfig() if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) @@ -188,7 +188,7 @@ func (h *confHandler) updateConfig(cfg *config.Config, key string, value any) er case "keyspace": return h.updateKeyspaceConfig(cfg, kp[len(kp)-1], value) case "micro-service": - return h.updateMicroServiceConfig(cfg, kp[len(kp)-1], value) + return h.updateMicroserviceConfig(cfg, kp[len(kp)-1], value) } return errors.Errorf("config prefix %s not found", kp[0]) } @@ -209,8 +209,8 @@ func (h *confHandler) updateKeyspaceConfig(config *config.Config, key string, va return err } -func (h *confHandler) updateMicroServiceConfig(config *config.Config, key string, value any) error { - updated, found, err := jsonutil.AddKeyValue(&config.MicroService, key, value) +func (h *confHandler) updateMicroserviceConfig(config *config.Config, key string, value any) error { + updated, found, err := jsonutil.AddKeyValue(&config.Microservice, key, value) if err != nil { return err } @@ -220,7 +220,7 @@ func (h *confHandler) updateMicroServiceConfig(config *config.Config, key string } if updated { - err = h.svr.SetMicroServiceConfig(config.MicroService) + err = h.svr.SetMicroserviceConfig(config.Microservice) } return err } @@ -339,7 +339,7 @@ func getConfigMap(cfg map[string]any, key []string, value any) map[string]any { // @Router /config/schedule [get] func (h *confHandler) GetScheduleConfig(w http.ResponseWriter, r *http.Request) { if h.svr.IsServiceIndependent(constant.SchedulingServiceName) && - r.Header.Get(apiutil.XForbiddenForwardToMicroServiceHeader) != "true" { + r.Header.Get(apiutil.XForbiddenForwardToMicroserviceHeader) != "true" { cfg, err := h.getSchedulingServerConfig() if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) @@ -412,7 +412,7 @@ func (h *confHandler) SetScheduleConfig(w http.ResponseWriter, r *http.Request) // @Router /config/replicate [get] func (h *confHandler) GetReplicationConfig(w http.ResponseWriter, r *http.Request) { if h.svr.IsServiceIndependent(constant.SchedulingServiceName) && - r.Header.Get(apiutil.XForbiddenForwardToMicroServiceHeader) != "true" { + r.Header.Get(apiutil.XForbiddenForwardToMicroserviceHeader) != "true" { cfg, err := h.getSchedulingServerConfig() if err != nil { h.rd.JSON(w, http.StatusInternalServerError, err.Error()) diff --git a/server/apiv2/handlers/micro_service.go b/server/apiv2/handlers/micro_service.go index b4d3d6bbe89..3a0d6a2c2a6 100644 --- a/server/apiv2/handlers/micro_service.go +++ b/server/apiv2/handlers/micro_service.go @@ -24,8 +24,8 @@ import ( "github.com/tikv/pd/server/apiv2/middlewares" ) -// RegisterMicroService registers microservice handler to the router. -func RegisterMicroService(r *gin.RouterGroup) { +// RegisterMicroservice registers microservice handler to the router. +func RegisterMicroservice(r *gin.RouterGroup) { router := r.Group("ms") router.GET("members/:service", GetMembers) router.GET("primary/:service", GetPrimary) @@ -40,7 +40,7 @@ func RegisterMicroService(r *gin.RouterGroup) { func GetMembers(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) if !svr.IsPDServiceMode() { - c.AbortWithStatusJSON(http.StatusNotFound, "not support micro service") + c.AbortWithStatusJSON(http.StatusNotFound, "not support microservice") return } @@ -66,7 +66,7 @@ func GetMembers(c *gin.Context) { func GetPrimary(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) if !svr.IsPDServiceMode() { - c.AbortWithStatusJSON(http.StatusNotFound, "not support micro service") + c.AbortWithStatusJSON(http.StatusNotFound, "not support microservice") return } diff --git a/server/apiv2/router.go b/server/apiv2/router.go index 7f927dd9705..8df34e0a742 100644 --- a/server/apiv2/router.go +++ b/server/apiv2/router.go @@ -56,6 +56,6 @@ func NewV2Handler(_ context.Context, svr *server.Server) (http.Handler, apiutil. root.GET("ready", handlers.Ready) handlers.RegisterKeyspace(root) handlers.RegisterTSOKeyspaceGroup(root) - handlers.RegisterMicroService(root) + handlers.RegisterMicroservice(root) return router, group, nil } diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index e482b7b8b68..4ad97dfd0cc 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -406,7 +406,7 @@ func (c *RaftCluster) Start(s Server, bootstrap bool) (err error) { func (c *RaftCluster) checkSchedulingService() { if c.isPDServiceMode { servers, err := discovery.Discover(c.etcdClient, constant.SchedulingServiceName) - if c.opt.GetMicroServiceConfig().IsSchedulingFallbackEnabled() && (err != nil || len(servers) == 0) { + if c.opt.GetMicroserviceConfig().IsSchedulingFallbackEnabled() && (err != nil || len(servers) == 0) { c.startSchedulingJobs(c, c.hbstreams) c.UnsetServiceIndependent(constant.SchedulingServiceName) } else { @@ -426,7 +426,7 @@ func (c *RaftCluster) checkSchedulingService() { // checkTSOService checks the TSO service. func (c *RaftCluster) checkTSOService() { if c.isPDServiceMode { - if c.opt.GetMicroServiceConfig().IsTSODynamicSwitchingEnabled() { + if c.opt.GetMicroserviceConfig().IsTSODynamicSwitchingEnabled() { servers, err := discovery.Discover(c.etcdClient, constant.TSOServiceName) if err != nil || len(servers) == 0 { if err := c.startTSOJobsIfNeeded(); err != nil { diff --git a/server/config/config.go b/server/config/config.go index 69cd76409bc..3d0d16bc376 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -162,7 +162,7 @@ type Config struct { Keyspace KeyspaceConfig `toml:"keyspace" json:"keyspace"` - MicroService MicroServiceConfig `toml:"micro-service" json:"micro-service"` + Microservice MicroserviceConfig `toml:"micro-service" json:"micro-service"` Controller rm.ControllerConfig `toml:"controller" json:"controller"` } @@ -457,7 +457,7 @@ func (c *Config) Adjust(meta *toml.MetaData, reloading bool) error { c.Keyspace.adjust(configMetaData.Child("keyspace")) - c.MicroService.adjust(configMetaData.Child("micro-service")) + c.Microservice.adjust(configMetaData.Child("micro-service")) if err := c.Security.Encryption.Adjust(); err != nil { return err @@ -839,13 +839,13 @@ func (c *DRAutoSyncReplicationConfig) adjust(meta *configutil.ConfigMetaData) { } } -// MicroServiceConfig is the configuration for micro service. -type MicroServiceConfig struct { +// MicroserviceConfig is the configuration for microservice. +type MicroserviceConfig struct { EnableSchedulingFallback bool `toml:"enable-scheduling-fallback" json:"enable-scheduling-fallback,string"` EnableTSODynamicSwitching bool `toml:"enable-tso-dynamic-switching" json:"enable-tso-dynamic-switching,string"` } -func (c *MicroServiceConfig) adjust(meta *configutil.ConfigMetaData) { +func (c *MicroserviceConfig) adjust(meta *configutil.ConfigMetaData) { if !meta.IsDefined("enable-scheduling-fallback") { c.EnableSchedulingFallback = defaultEnableSchedulingFallback } @@ -854,19 +854,19 @@ func (c *MicroServiceConfig) adjust(meta *configutil.ConfigMetaData) { } } -// Clone returns a copy of micro service config. -func (c *MicroServiceConfig) Clone() *MicroServiceConfig { +// Clone returns a copy of microservice config. +func (c *MicroserviceConfig) Clone() *MicroserviceConfig { cfg := *c return &cfg } // IsSchedulingFallbackEnabled returns whether to enable scheduling service fallback to PD service. -func (c *MicroServiceConfig) IsSchedulingFallbackEnabled() bool { +func (c *MicroserviceConfig) IsSchedulingFallbackEnabled() bool { return c.EnableSchedulingFallback } // IsTSODynamicSwitchingEnabled returns whether to enable TSO dynamic switching. -func (c *MicroServiceConfig) IsTSODynamicSwitchingEnabled() bool { +func (c *MicroserviceConfig) IsTSODynamicSwitchingEnabled() bool { return c.EnableTSODynamicSwitching } diff --git a/server/config/persist_options.go b/server/config/persist_options.go index 59d42383743..5ce24ac3f56 100644 --- a/server/config/persist_options.go +++ b/server/config/persist_options.go @@ -55,7 +55,7 @@ type PersistOptions struct { replicationMode atomic.Value labelProperty atomic.Value keyspace atomic.Value - microService atomic.Value + microservice atomic.Value storeConfig atomic.Value clusterVersion unsafe.Pointer } @@ -69,7 +69,7 @@ func NewPersistOptions(cfg *Config) *PersistOptions { o.replicationMode.Store(&cfg.ReplicationMode) o.labelProperty.Store(cfg.LabelProperty) o.keyspace.Store(&cfg.Keyspace) - o.microService.Store(&cfg.MicroService) + o.microservice.Store(&cfg.Microservice) // storeConfig will be fetched from TiKV later, // set it to an empty config here first. o.storeConfig.Store(&sc.StoreConfig{}) @@ -138,14 +138,14 @@ func (o *PersistOptions) SetKeyspaceConfig(cfg *KeyspaceConfig) { o.keyspace.Store(cfg) } -// GetMicroServiceConfig returns the micro service configuration. -func (o *PersistOptions) GetMicroServiceConfig() *MicroServiceConfig { - return o.microService.Load().(*MicroServiceConfig) +// GetMicroserviceConfig returns the microservice configuration. +func (o *PersistOptions) GetMicroserviceConfig() *MicroserviceConfig { + return o.microservice.Load().(*MicroserviceConfig) } -// SetMicroServiceConfig sets the micro service configuration. -func (o *PersistOptions) SetMicroServiceConfig(cfg *MicroServiceConfig) { - o.microService.Store(cfg) +// SetMicroserviceConfig sets the microservice configuration. +func (o *PersistOptions) SetMicroserviceConfig(cfg *MicroserviceConfig) { + o.microservice.Store(cfg) } // GetStoreConfig returns the store config. @@ -791,7 +791,7 @@ func (o *PersistOptions) Persist(storage endpoint.ConfigStorage) error { ReplicationMode: *o.GetReplicationModeConfig(), LabelProperty: o.GetLabelPropertyConfig(), Keyspace: *o.GetKeyspaceConfig(), - MicroService: *o.GetMicroServiceConfig(), + Microservice: *o.GetMicroserviceConfig(), ClusterVersion: *o.GetClusterVersion(), }, StoreConfig: *o.GetStoreConfig(), @@ -825,7 +825,7 @@ func (o *PersistOptions) Reload(storage endpoint.ConfigStorage) error { o.replicationMode.Store(&cfg.ReplicationMode) o.labelProperty.Store(cfg.LabelProperty) o.keyspace.Store(&cfg.Keyspace) - o.microService.Store(&cfg.MicroService) + o.microservice.Store(&cfg.Microservice) o.storeConfig.Store(&cfg.StoreConfig) o.SetClusterVersion(&cfg.ClusterVersion) } diff --git a/server/server.go b/server/server.go index 8de2a5c57f3..7ecf309b1d9 100644 --- a/server/server.go +++ b/server/server.go @@ -300,7 +300,7 @@ func CreateServer(ctx context.Context, cfg *config.Config, services []string, le }) s.registry.RegisterService("MetaStorage", ms_server.NewService) s.registry.RegisterService("ResourceManager", rm_server.NewService[*Server]) - // Register the micro services REST path. + // Register the microservices REST path. s.registry.InstallAllRESTHandler(s, etcdCfg.UserHandlers) etcdCfg.ServiceRegister = func(gs *grpc.Server) { @@ -308,7 +308,7 @@ func CreateServer(ctx context.Context, cfg *config.Config, services []string, le pdpb.RegisterPDServer(gs, grpcServer) keyspacepb.RegisterKeyspaceServer(gs, &KeyspaceServer{GrpcServer: grpcServer}) diagnosticspb.RegisterDiagnosticsServer(gs, s) - // Register the micro services GRPC service. + // Register the microservices GRPC service. s.registry.InstallAllGRPCServices(s, gs) s.grpcServer = gs } @@ -948,7 +948,7 @@ func (s *Server) GetConfig() *config.Config { cfg.PDServerCfg = *s.persistOptions.GetPDServerConfig().Clone() cfg.ReplicationMode = *s.persistOptions.GetReplicationModeConfig() cfg.Keyspace = *s.persistOptions.GetKeyspaceConfig().Clone() - cfg.MicroService = *s.persistOptions.GetMicroServiceConfig().Clone() + cfg.Microservice = *s.persistOptions.GetMicroserviceConfig().Clone() cfg.LabelProperty = s.persistOptions.GetLabelPropertyConfig().Clone() cfg.ClusterVersion = *s.persistOptions.GetClusterVersion() return cfg @@ -979,24 +979,24 @@ func (s *Server) SetKeyspaceConfig(cfg config.KeyspaceConfig) error { return nil } -// GetMicroServiceConfig gets the micro service config information. -func (s *Server) GetMicroServiceConfig() *config.MicroServiceConfig { - return s.persistOptions.GetMicroServiceConfig().Clone() +// GetMicroserviceConfig gets the microservice config information. +func (s *Server) GetMicroserviceConfig() *config.MicroserviceConfig { + return s.persistOptions.GetMicroserviceConfig().Clone() } -// SetMicroServiceConfig sets the micro service config information. -func (s *Server) SetMicroServiceConfig(cfg config.MicroServiceConfig) error { - old := s.persistOptions.GetMicroServiceConfig() - s.persistOptions.SetMicroServiceConfig(&cfg) +// SetMicroserviceConfig sets the microservice config information. +func (s *Server) SetMicroserviceConfig(cfg config.MicroserviceConfig) error { + old := s.persistOptions.GetMicroserviceConfig() + s.persistOptions.SetMicroserviceConfig(&cfg) if err := s.persistOptions.Persist(s.storage); err != nil { - s.persistOptions.SetMicroServiceConfig(old) - log.Error("failed to update micro service config", + s.persistOptions.SetMicroserviceConfig(old) + log.Error("failed to update microservice config", zap.Reflect("new", cfg), zap.Reflect("old", old), errs.ZapError(err)) return err } - log.Info("micro service config is updated", zap.Reflect("new", cfg), zap.Reflect("old", old)) + log.Info("microservice config is updated", zap.Reflect("new", cfg), zap.Reflect("old", old)) return nil } @@ -1391,7 +1391,7 @@ func (s *Server) GetRaftCluster() *cluster.RaftCluster { // IsServiceIndependent returns whether the service is independent. func (s *Server) IsServiceIndependent(name string) bool { if s.mode == PDServiceMode && !s.IsClosed() { - if name == constant.TSOServiceName && !s.GetMicroServiceConfig().IsTSODynamicSwitchingEnabled() { + if name == constant.TSOServiceName && !s.GetMicroserviceConfig().IsTSODynamicSwitchingEnabled() { return true } return s.cluster.IsServiceIndependent(name) diff --git a/tests/integrations/mcs/members/member_test.go b/tests/integrations/mcs/members/member_test.go index 7e83ea570b9..0ec0b323d7a 100644 --- a/tests/integrations/mcs/members/member_test.go +++ b/tests/integrations/mcs/members/member_test.go @@ -126,29 +126,29 @@ func (suite *memberTestSuite) TearDownTest() { func (suite *memberTestSuite) TestMembers() { re := suite.Require() - members, err := suite.pdClient.GetMicroServiceMembers(suite.ctx, "tso") + members, err := suite.pdClient.GetMicroserviceMembers(suite.ctx, "tso") re.NoError(err) re.Len(members, 3) - members, err = suite.pdClient.GetMicroServiceMembers(suite.ctx, "scheduling") + members, err = suite.pdClient.GetMicroserviceMembers(suite.ctx, "scheduling") re.NoError(err) re.Len(members, 3) } func (suite *memberTestSuite) TestPrimary() { re := suite.Require() - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, "tso") + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, "tso") re.NoError(err) re.NotEmpty(primary) - primary, err = suite.pdClient.GetMicroServicePrimary(suite.ctx, "scheduling") + primary, err = suite.pdClient.GetMicroservicePrimary(suite.ctx, "scheduling") re.NoError(err) re.NotEmpty(primary) } func (suite *memberTestSuite) TestPrimaryWorkWhileOtherServerClose() { re := suite.Require() - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, "tso") + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, "tso") re.NoError(err) re.NotEmpty(primary) @@ -162,7 +162,7 @@ func (suite *memberTestSuite) TestPrimaryWorkWhileOtherServerClose() { nodes = suite.schedulingNodes } - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) // Close non-primary node. @@ -174,7 +174,7 @@ func (suite *memberTestSuite) TestPrimaryWorkWhileOtherServerClose() { tests.WaitForPrimaryServing(re, nodes) // primary should be same with before. - curPrimary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + curPrimary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) re.Equal(primary, curPrimary) } @@ -193,7 +193,7 @@ func (suite *memberTestSuite) TestTransferPrimary() { } // Test resign primary by random - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) newPrimaryData := make(map[string]any) @@ -214,7 +214,7 @@ func (suite *memberTestSuite) TestTransferPrimary() { return false }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - primary, err = suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err = suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) // Test transfer primary to a specific node @@ -240,7 +240,7 @@ func (suite *memberTestSuite) TestTransferPrimary() { return nodes[newPrimary].IsServing() }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) - primary, err = suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err = suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) re.Equal(primary, newPrimary) @@ -268,7 +268,7 @@ func (suite *memberTestSuite) TestCampaignPrimaryAfterTransfer() { nodes = suite.schedulingNodes } - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) // Test transfer primary to a specific node @@ -292,7 +292,7 @@ func (suite *memberTestSuite) TestCampaignPrimaryAfterTransfer() { resp.Body.Close() tests.WaitForPrimaryServing(re, nodes) - newPrimary, err = suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + newPrimary, err = suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) re.NotEqual(primary, newPrimary) @@ -300,7 +300,7 @@ func (suite *memberTestSuite) TestCampaignPrimaryAfterTransfer() { nodes[newPrimary].Close() tests.WaitForPrimaryServing(re, nodes) // Primary should be different with before - anotherPrimary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + anotherPrimary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) re.NotEqual(newPrimary, anotherPrimary) } @@ -308,7 +308,7 @@ func (suite *memberTestSuite) TestCampaignPrimaryAfterTransfer() { func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpired() { re := suite.Require() - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, "tso") + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, "tso") re.NoError(err) re.NotEmpty(primary) @@ -322,7 +322,7 @@ func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpired() { nodes = suite.schedulingNodes } - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) // Test transfer primary to a specific node @@ -363,7 +363,7 @@ func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpired() { // TestTransferPrimaryWhileLeaseExpiredAndServerDown tests transfer primary while lease expired and server down func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpiredAndServerDown() { re := suite.Require() - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, "tso") + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, "tso") re.NoError(err) re.NotEmpty(primary) @@ -377,7 +377,7 @@ func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpiredAndServerDown( nodes = suite.schedulingNodes } - primary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + primary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) // Test transfer primary to a specific node @@ -415,7 +415,7 @@ func (suite *memberTestSuite) TestTransferPrimaryWhileLeaseExpiredAndServerDown( tests.WaitForPrimaryServing(re, nodes) // Primary should be different with before - onlyPrimary, err := suite.pdClient.GetMicroServicePrimary(suite.ctx, service) + onlyPrimary, err := suite.pdClient.GetMicroservicePrimary(suite.ctx, service) re.NoError(err) re.NotEqual(newPrimary, onlyPrimary) } diff --git a/tests/integrations/mcs/scheduling/api_test.go b/tests/integrations/mcs/scheduling/api_test.go index 9b6b3d95145..e52ced10011 100644 --- a/tests/integrations/mcs/scheduling/api_test.go +++ b/tests/integrations/mcs/scheduling/api_test.go @@ -118,29 +118,29 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { // Test operators err := testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators"), &respSlice, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) re.Empty(respSlice) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators"), []byte(``), - testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators/2"), nil, - testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators/2"), - testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators/records"), nil, - testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // Test checker err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "checker/merge"), &resp, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) re.False(resp["paused"].(bool)) @@ -151,7 +151,7 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { pauseArgs, err := json.Marshal(input) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "checker/merge"), pauseArgs, - testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) } postChecker(30) @@ -170,7 +170,7 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { // "/schedulers/{name}", http.MethodDelete testutil.Eventually(re, func() bool { err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers"), &respSlice, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) return slice.Contains(respSlice, "balance-leader-scheduler") }) @@ -181,18 +181,18 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { pauseArgs, err := json.Marshal(input) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers/balance-leader-scheduler"), pauseArgs, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) } postScheduler(30) postScheduler(0) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers/diagnostic/balance-leader-scheduler"), &resp, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "scheduler-config"), &resp, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) re.Contains(resp, "balance-leader-scheduler") re.Contains(resp, "balance-hot-region-scheduler") @@ -203,16 +203,16 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { } for _, schedulerName := range schedulers { err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s/%s/%s", urlPrefix, "scheduler-config", schedulerName, "list"), &resp, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) } err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers"), nil, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers/balance-leader-scheduler"), - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) input := make(map[string]any) @@ -220,74 +220,74 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { b, err := json.Marshal(input) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "schedulers"), b, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) // Test hotspot var hotRegions statistics.StoreHotPeersInfos err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "hotspot/regions/write"), &hotRegions, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "hotspot/regions/read"), &hotRegions, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) var stores handler.HotStoreStats err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "hotspot/stores"), &stores, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) var buckets handler.HotBucketsResponse err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "hotspot/buckets"), &buckets, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) var history storage.HistoryHotRegions err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "hotspot/regions/history"), &history, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // Test region label var labelRules []*labeler.LabelRule err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/region-label/rules"), &labelRules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.ReadGetJSONWithBody(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/region-label/rules/ids"), []byte(`["rule1", "rule3"]`), - &labelRules, testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + &labelRules, testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/region-label/rule/rule1"), nil, - testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "region/id/1"), nil, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "region/id/1/label/key"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "region/id/1/labels"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // Test Region body := fmt.Sprintf(`{"start_key":"%s", "end_key": "%s"}`, hex.EncodeToString([]byte("a1")), hex.EncodeToString([]byte("a3"))) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "regions/accelerate-schedule"), []byte(body), - testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) body = fmt.Sprintf(`[{"start_key":"%s", "end_key": "%s"}, {"start_key":"%s", "end_key": "%s"}]`, hex.EncodeToString([]byte("a1")), hex.EncodeToString([]byte("a3")), hex.EncodeToString([]byte("a4")), hex.EncodeToString([]byte("a6"))) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "regions/accelerate-schedule/batch"), []byte(body), - testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) body = fmt.Sprintf(`{"start_key":"%s", "end_key": "%s"}`, hex.EncodeToString([]byte("b1")), hex.EncodeToString([]byte("b3"))) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "regions/scatter"), []byte(body), - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) body = fmt.Sprintf(`{"retry_limit":%v, "split_keys": ["%s","%s","%s"]}`, 3, hex.EncodeToString([]byte("bbb")), hex.EncodeToString([]byte("ccc")), hex.EncodeToString([]byte("ddd"))) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "regions/split"), []byte(body), - testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf(`%s/regions/replicated?startKey=%s&endKey=%s`, urlPrefix, hex.EncodeToString([]byte("a1")), hex.EncodeToString([]byte("a2"))), nil, - testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // Test rules: only forward `GET` request var rules []*placement.Rule @@ -305,76 +305,76 @@ func (suite *apiTestSuite) checkAPIForward(cluster *tests.TestCluster) { re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules"), &rules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules/batch"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules/group/pd"), &rules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules/region/2"), &rules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) var fit placement.RegionFit err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules/region/2/detail"), &fit, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules/key/0000000000000001"), &rules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule/pd/2"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule/pd/2"), - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule_group/pd"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule_group/pd"), - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule_group"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rule_groups"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule/pd"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) err = testutil.CheckDelete(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule/pd"), - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule/pd"), rulesArgs, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) // test redirect is disabled err = testutil.CheckGetJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule/pd"), nil, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) req, err := http.NewRequest(http.MethodGet, fmt.Sprintf("%s/%s", urlPrefix, "config/placement-rule/pd"), http.NoBody) re.NoError(err) - req.Header.Set(apiutil.XForbiddenForwardToMicroServiceHeader, "true") + req.Header.Set(apiutil.XForbiddenForwardToMicroserviceHeader, "true") httpResp, err := tests.TestDialClient.Do(req) re.NoError(err) re.Equal(http.StatusOK, httpResp.StatusCode) defer httpResp.Body.Close() - re.Empty(httpResp.Header.Get(apiutil.XForwardedToMicroServiceHeader)) + re.Empty(httpResp.Header.Get(apiutil.XForwardedToMicroserviceHeader)) } func (suite *apiTestSuite) TestConfig() { @@ -536,14 +536,14 @@ func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { // follower will forward to scheduling server directly re.NotEqual(cluster.GetLeaderServer().GetAddr(), followerAddr) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules"), &rules, - testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true"), + testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true"), ) re.NoError(err) } else { // follower will forward to leader server re.NotEqual(cluster.GetLeaderServer().GetAddr(), followerAddr) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config/rules"), &rules, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader), + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader), ) re.NoError(err) } @@ -552,7 +552,7 @@ func (suite *apiTestSuite) checkFollowerForward(cluster *tests.TestCluster) { re.NotEqual(cluster.GetLeaderServer().GetAddr(), followerAddr) results := make(map[string]any) err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "config"), &results, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader), + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader), ) re.NoError(err) } diff --git a/tests/integrations/mcs/scheduling/server_test.go b/tests/integrations/mcs/scheduling/server_test.go index 9a3d33d1dcf..7271111bd79 100644 --- a/tests/integrations/mcs/scheduling/server_test.go +++ b/tests/integrations/mcs/scheduling/server_test.go @@ -216,10 +216,10 @@ func (suite *serverTestSuite) TestForwardStoreHeartbeat() { func (suite *serverTestSuite) TestSchedulingServiceFallback() { re := suite.Require() leaderServer := suite.pdLeader.GetServer() - conf := leaderServer.GetMicroServiceConfig().Clone() + conf := leaderServer.GetMicroserviceConfig().Clone() // Change back to the default value. conf.EnableSchedulingFallback = true - leaderServer.SetMicroServiceConfig(*conf) + leaderServer.SetMicroserviceConfig(*conf) // PD service will execute scheduling jobs since there is no scheduling server. testutil.Eventually(re, func() bool { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() @@ -267,15 +267,15 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { }) leaderServer := suite.pdLeader.GetServer() // After Disabling scheduling service fallback, the PD service will stop scheduling. - conf := leaderServer.GetMicroServiceConfig().Clone() + conf := leaderServer.GetMicroserviceConfig().Clone() conf.EnableSchedulingFallback = false - leaderServer.SetMicroServiceConfig(*conf) + leaderServer.SetMicroserviceConfig(*conf) testutil.Eventually(re, func() bool { return !suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) // Enable scheduling service fallback again, the PD service will restart scheduling. conf.EnableSchedulingFallback = true - leaderServer.SetMicroServiceConfig(*conf) + leaderServer.SetMicroserviceConfig(*conf) testutil.Eventually(re, func() bool { return suite.pdLeader.GetServer().GetRaftCluster().IsSchedulingControllerRunning() }) @@ -294,7 +294,7 @@ func (suite *serverTestSuite) TestDisableSchedulingServiceFallback() { }) // Disable scheduling service fallback and stop scheduling server. PD service won't execute scheduling jobs again. conf.EnableSchedulingFallback = false - leaderServer.SetMicroServiceConfig(*conf) + leaderServer.SetMicroserviceConfig(*conf) tc.GetPrimaryServer().Close() time.Sleep(time.Second) testutil.Eventually(re, func() bool { diff --git a/tests/integrations/mcs/tso/api_test.go b/tests/integrations/mcs/tso/api_test.go index dceb5ccdf7c..97f56cbcee1 100644 --- a/tests/integrations/mcs/tso/api_test.go +++ b/tests/integrations/mcs/tso/api_test.go @@ -107,13 +107,13 @@ func (suite *tsoAPITestSuite) TestForwardResetTS() { // Test reset ts input := []byte(`{"tso":"121312", "force-use-larger":true}`) err := testutil.CheckPostJSON(tests.TestDialClient, url, input, - testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully"), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully"), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // Test reset ts with invalid tso input = []byte(`{}`) err = testutil.CheckPostJSON(tests.TestDialClient, url, input, - testutil.StatusNotOK(re), testutil.StringContain(re, "invalid tso value"), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusNotOK(re), testutil.StringContain(re, "invalid tso value"), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) } @@ -217,14 +217,14 @@ func TestForwardOnlyTSONoScheduling(t *testing.T) { // Test /operators, it should not forward when there is no scheduling server. var slice []string err = testutil.ReadGetJSON(re, tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "operators"), &slice, - testutil.WithoutHeader(re, apiutil.XForwardedToMicroServiceHeader)) + testutil.WithoutHeader(re, apiutil.XForwardedToMicroserviceHeader)) re.NoError(err) re.Empty(slice) // Test admin/reset-ts, it should forward to tso server. input := []byte(`{"tso":"121312", "force-use-larger":true}`) err = testutil.CheckPostJSON(tests.TestDialClient, fmt.Sprintf("%s/%s", urlPrefix, "admin/reset-ts"), input, - testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully"), testutil.WithHeader(re, apiutil.XForwardedToMicroServiceHeader, "true")) + testutil.StatusOK(re), testutil.StringContain(re, "Reset ts successfully"), testutil.WithHeader(re, apiutil.XForwardedToMicroserviceHeader, "true")) re.NoError(err) // If close tso server, it should try forward to tso server, but return error in pd service mode. diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index 7416a314949..4c9bb4248e5 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -169,7 +169,7 @@ func checkTSOPath(re *require.Assertions, isPDServiceMode bool) { defer cancel() if isPDServiceMode { cluster, err = tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { - conf.MicroService.EnableTSODynamicSwitching = false + conf.Microservice.EnableTSODynamicSwitching = false }) } else { cluster, err = tests.NewTestCluster(ctx, 1) @@ -276,9 +276,9 @@ func TestForwardTSORelated(t *testing.T) { suite := NewPDServiceForward(re) defer suite.ShutDown() leaderServer := suite.cluster.GetLeaderServer().GetServer() - cfg := leaderServer.GetMicroServiceConfig().Clone() + cfg := leaderServer.GetMicroserviceConfig().Clone() cfg.EnableTSODynamicSwitching = false - leaderServer.SetMicroServiceConfig(*cfg) + leaderServer.SetMicroserviceConfig(*cfg) // Unable to use the tso-related interface without tso server suite.checkUnavailableTSO(re) tc, err := tests.NewTestTSOCluster(suite.ctx, 1, suite.backendEndpoints) @@ -600,7 +600,7 @@ func TestTSOServiceSwitch(t *testing.T) { tc, err := tests.NewTestPDServiceCluster(ctx, 1, func(conf *config.Config, _ string) { - conf.MicroService.EnableTSODynamicSwitching = true + conf.Microservice.EnableTSODynamicSwitching = true }, ) re.NoError(err) @@ -638,9 +638,9 @@ func TestTSOServiceSwitch(t *testing.T) { re.NoError(err) // Disable TSO switching - cfg := pdLeader.GetServer().GetMicroServiceConfig().Clone() + cfg := pdLeader.GetServer().GetMicroserviceConfig().Clone() cfg.EnableTSODynamicSwitching = false - pdLeader.GetServer().SetMicroServiceConfig(*cfg) + pdLeader.GetServer().SetMicroserviceConfig(*cfg) tsoCluster.Destroy() @@ -654,10 +654,10 @@ func TestTSOServiceSwitch(t *testing.T) { } // Now enable TSO switching - cfg = pdLeader.GetServer().GetMicroServiceConfig().Clone() + cfg = pdLeader.GetServer().GetMicroserviceConfig().Clone() cfg.EnableTSODynamicSwitching = true - pdLeader.GetServer().SetMicroServiceConfig(*cfg) + pdLeader.GetServer().SetMicroserviceConfig(*cfg) // Wait for PD to detect the change time.Sleep(300 * time.Millisecond) diff --git a/tests/integrations/tso/client_test.go b/tests/integrations/tso/client_test.go index a06e44ed4ab..87ddd9e226b 100644 --- a/tests/integrations/tso/client_test.go +++ b/tests/integrations/tso/client_test.go @@ -99,7 +99,7 @@ func (suite *tsoClientTestSuite) SetupSuite() { suite.cluster, err = tests.NewTestCluster(suite.ctx, serverCount) } else { suite.cluster, err = tests.NewTestPDServiceCluster(suite.ctx, serverCount, func(conf *config.Config, _ string) { - conf.MicroService.EnableTSODynamicSwitching = false + conf.Microservice.EnableTSODynamicSwitching = false }) } re.NoError(err) diff --git a/tools/pd-ctl/pdctl/command/config_command.go b/tools/pd-ctl/pdctl/command/config_command.go index f89c63bc51c..c49b0e3b9e5 100644 --- a/tools/pd-ctl/pdctl/command/config_command.go +++ b/tools/pd-ctl/pdctl/command/config_command.go @@ -81,7 +81,7 @@ func NewShowConfigCommand() *cobra.Command { sc.AddCommand(newShowReplicationModeCommand()) sc.AddCommand(NewShowServerConfigCommand()) sc.AddCommand(NewShowServiceMiddlewareConfigCommand()) - sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") return sc } @@ -92,7 +92,7 @@ func NewShowAllConfigCommand() *cobra.Command { Short: "show all config of PD", Run: showAllConfigCommandFunc, } - sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") return sc } @@ -103,7 +103,7 @@ func NewShowScheduleConfigCommand() *cobra.Command { Short: "show schedule config of PD", Run: showScheduleConfigCommandFunc, } - sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") return sc } @@ -114,7 +114,7 @@ func NewShowReplicationConfigCommand() *cobra.Command { Short: "show replication config of PD", Run: showReplicationConfigCommandFunc, } - sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + sc.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") return sc } @@ -528,7 +528,7 @@ func NewPlacementRulesCommand() *cobra.Command { show.Flags().String("id", "", "rule id") show.Flags().String("region", "", "region id") show.Flags().Bool("detail", false, "detailed match info for region") - show.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + show.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") load := &cobra.Command{ Use: "load", Short: "load placement rules to a file", @@ -538,7 +538,7 @@ func NewPlacementRulesCommand() *cobra.Command { load.Flags().String("id", "", "rule id") load.Flags().String("region", "", "region id") load.Flags().String("out", "rules.json", "the filename contains rules") - load.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + load.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") save := &cobra.Command{ Use: "save", Short: "save rules from file", @@ -554,7 +554,7 @@ func NewPlacementRulesCommand() *cobra.Command { Short: "show rule group configuration(s)", Run: showRuleGroupFunc, } - ruleGroupShow.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + ruleGroupShow.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") ruleGroupSet := &cobra.Command{ Use: "set ", Short: "update rule group configuration", @@ -577,7 +577,7 @@ func NewPlacementRulesCommand() *cobra.Command { Run: getRuleBundle, } ruleBundleGet.Flags().String("out", "", "the output file") - ruleBundleGet.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + ruleBundleGet.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") ruleBundleSet := &cobra.Command{ Use: "set", Short: "set rule group config and its rules from file", @@ -596,7 +596,7 @@ func NewPlacementRulesCommand() *cobra.Command { Run: loadRuleBundle, } ruleBundleLoad.Flags().String("out", "rules.json", "the output file") - ruleBundleLoad.Flags().Bool(flagFromPDService, false, "read data from PD service rather than micro service") + ruleBundleLoad.Flags().Bool(flagFromPDService, false, "read data from PD service rather than microservice") ruleBundleSave := &cobra.Command{ Use: "save", Short: "save all group configs and rules from file", @@ -895,9 +895,9 @@ func saveRuleBundle(cmd *cobra.Command, _ []string) { func buildHeader(cmd *cobra.Command) http.Header { header := http.Header{} - forbiddenRedirectToMicroService, err := cmd.Flags().GetBool(flagFromPDService) - if err == nil && forbiddenRedirectToMicroService { - header.Add(apiutil.XForbiddenForwardToMicroServiceHeader, "true") + forbiddenRedirectToMicroservice, err := cmd.Flags().GetBool(flagFromPDService) + if err == nil && forbiddenRedirectToMicroservice { + header.Add(apiutil.XForbiddenForwardToMicroserviceHeader, "true") } return header } diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index ce79277db1b..4e91f0de514 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -207,9 +207,9 @@ func dial(req *http.Request) (string, error) { if err != nil { return "", err } - if req.Header.Get(apiutil.XForbiddenForwardToMicroServiceHeader) == "true" { - if resp.Header.Get(apiutil.XForwardedToMicroServiceHeader) == "true" { - return string(content), errors.Errorf("the request is forwarded to micro service unexpectedly") + if req.Header.Get(apiutil.XForbiddenForwardToMicroserviceHeader) == "true" { + if resp.Header.Get(apiutil.XForwardedToMicroserviceHeader) == "true" { + return string(content), errors.Errorf("the request is forwarded to microservice unexpectedly") } } return string(content), nil diff --git a/tools/pd-ctl/tests/config/config_test.go b/tools/pd-ctl/tests/config/config_test.go index 39820a6c7b7..3b622beecbf 100644 --- a/tools/pd-ctl/tests/config/config_test.go +++ b/tools/pd-ctl/tests/config/config_test.go @@ -1233,11 +1233,11 @@ func (suite *configTestSuite) checkPDServerConfig(cluster *pdTests.TestCluster) re.Equal(int(3), conf.FlowRoundByDigit) } -func (suite *configTestSuite) TestMicroServiceConfig() { - suite.env.RunTestBasedOnMode(suite.checkMicroServiceConfig) +func (suite *configTestSuite) TestMicroserviceConfig() { + suite.env.RunTestBasedOnMode(suite.checkMicroserviceConfig) } -func (suite *configTestSuite) checkMicroServiceConfig(cluster *pdTests.TestCluster) { +func (suite *configTestSuite) checkMicroserviceConfig(cluster *pdTests.TestCluster) { re := suite.Require() leaderServer := cluster.GetLeaderServer() pdAddr := leaderServer.GetAddr() @@ -1254,13 +1254,13 @@ func (suite *configTestSuite) checkMicroServiceConfig(cluster *pdTests.TestClust re.NoError(err) cfg := config.Config{} re.NoError(json.Unmarshal(output, &cfg)) - re.True(svr.GetMicroServiceConfig().EnableSchedulingFallback) - re.True(cfg.MicroService.EnableSchedulingFallback) + re.True(svr.GetMicroserviceConfig().EnableSchedulingFallback) + re.True(cfg.Microservice.EnableSchedulingFallback) // config set enable-scheduling-fallback args := []string{"-u", pdAddr, "config", "set", "enable-scheduling-fallback", "false"} _, err = tests.ExecuteCommand(cmd, args...) re.NoError(err) - re.False(svr.GetMicroServiceConfig().EnableSchedulingFallback) + re.False(svr.GetMicroserviceConfig().EnableSchedulingFallback) } func (suite *configTestSuite) TestRegionRules() {