diff --git a/client/client.go b/client/client.go index 6225321e25b..f34f5897013 100644 --- a/client/client.go +++ b/client/client.go @@ -91,7 +91,7 @@ type Client interface { // client should retry later. GetRegion(ctx context.Context, key []byte, opts ...GetRegionOption) (*Region, error) // GetRegionFromMember gets a region from certain members. - GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string) (*Region, error) + GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string, opts ...GetRegionOption) (*Region, error) // GetPrevRegion gets the previous region and its leader Peer of the region where the key is located. GetPrevRegion(ctx context.Context, key []byte, opts ...GetRegionOption) (*Region, error) // GetRegionByID gets a region and its leader Peer from PD by id. @@ -100,7 +100,7 @@ type Client interface { // Limit limits the maximum number of regions returned. // If a region has no leader, corresponding leader will be placed by a peer // with empty value (PeerID is 0). - ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*Region, error) + ScanRegions(ctx context.Context, key, endKey []byte, limit int, opts ...GetRegionOption) ([]*Region, error) // GetStore gets a store from PD by store id. // The store may expire later. Caller is responsible for caching and taking care // of store change. @@ -200,7 +200,8 @@ func WithSkipStoreLimit() RegionsOption { // GetRegionOp represents available options when getting regions. type GetRegionOp struct { - needBuckets bool + needBuckets bool + allowFollowerHandle bool } // GetRegionOption configures GetRegionOp. @@ -211,6 +212,11 @@ func WithBuckets() GetRegionOption { return func(op *GetRegionOp) { op.needBuckets = true } } +// WithAllowFollowerHandle means that client can send request to follower and let it handle this request. +func WithAllowFollowerHandle() GetRegionOption { + return func(op *GetRegionOp) { op.allowFollowerHandle = true } +} + // LeaderHealthCheckInterval might be changed in the unit to shorten the testing time. var LeaderHealthCheckInterval = time.Second @@ -701,6 +707,12 @@ func (c *client) UpdateOption(option DynamicOption, value interface{}) error { return errors.New("[pd] invalid value type for EnableTSOFollowerProxy option, it should be bool") } c.option.setEnableTSOFollowerProxy(enable) + case EnableFollowerHandle: + enable, ok := value.(bool) + if !ok { + return errors.New("[pd] invalid value type for EnableFollowerHandle option, it should be bool") + } + c.option.setEnableFollowerHandle(enable) default: return errors.New("[pd] unsupported client option") } @@ -954,7 +966,7 @@ func isNetworkError(code codes.Code) bool { return code == codes.Unavailable || code == codes.DeadlineExceeded } -func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string) (*Region, error) { +func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs []string, opts ...GetRegionOption) (*Region, error) { if span := opentracing.SpanFromContext(ctx); span != nil { span = opentracing.StartSpan("pdclient.GetRegionFromMember", opentracing.ChildOf(span.Context())) defer span.Finish() @@ -1058,7 +1070,7 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...Get return handleRegionResponse(resp), nil } -func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*Region, error) { +func (c *client) ScanRegions(ctx context.Context, key, endKey []byte, limit int, opts ...GetRegionOption) ([]*Region, error) { if span := opentracing.SpanFromContext(ctx); span != nil { span = opentracing.StartSpan("pdclient.ScanRegions", opentracing.ChildOf(span.Context())) defer span.Finish() diff --git a/client/http/api.go b/client/http/api.go index 1826e2231ee..6b317330b61 100644 --- a/client/http/api.go +++ b/client/http/api.go @@ -23,19 +23,20 @@ import ( // The following constants are the paths of PD HTTP APIs. const ( // Metadata - HotRead = "/pd/api/v1/hotspot/regions/read" - HotWrite = "/pd/api/v1/hotspot/regions/write" - HotHistory = "/pd/api/v1/hotspot/regions/history" - RegionByIDPrefix = "/pd/api/v1/region/id" - regionByKey = "/pd/api/v1/region/key" - Regions = "/pd/api/v1/regions" - regionsByKey = "/pd/api/v1/regions/key" - RegionsByStoreIDPrefix = "/pd/api/v1/regions/store" - EmptyRegions = "/pd/api/v1/regions/check/empty-region" - AccelerateSchedule = "/pd/api/v1/regions/accelerate-schedule" - store = "/pd/api/v1/store" - Stores = "/pd/api/v1/stores" - StatsRegion = "/pd/api/v1/stats/region" + HotRead = "/pd/api/v1/hotspot/regions/read" + HotWrite = "/pd/api/v1/hotspot/regions/write" + HotHistory = "/pd/api/v1/hotspot/regions/history" + RegionByIDPrefix = "/pd/api/v1/region/id" + regionByKey = "/pd/api/v1/region/key" + Regions = "/pd/api/v1/regions" + regionsByKey = "/pd/api/v1/regions/key" + RegionsByStoreIDPrefix = "/pd/api/v1/regions/store" + EmptyRegions = "/pd/api/v1/regions/check/empty-region" + AccelerateSchedule = "/pd/api/v1/regions/accelerate-schedule" + AccelerateScheduleInBatch = "/pd/api/v1/regions/accelerate-schedule/batch" + store = "/pd/api/v1/store" + Stores = "/pd/api/v1/stores" + StatsRegion = "/pd/api/v1/stats/region" // Config Config = "/pd/api/v1/config" ClusterVersion = "/pd/api/v1/config/cluster-version" @@ -44,8 +45,11 @@ const ( // Rule PlacementRule = "/pd/api/v1/config/rule" PlacementRules = "/pd/api/v1/config/rules" + PlacementRulesInBatch = "/pd/api/v1/config/rules/batch" placementRulesByGroup = "/pd/api/v1/config/rules/group" PlacementRuleBundle = "/pd/api/v1/config/placement-rule" + placementRuleGroup = "/pd/api/v1/config/rule_group" + placementRuleGroups = "/pd/api/v1/config/rule_groups" RegionLabelRule = "/pd/api/v1/config/region-label/rule" RegionLabelRules = "/pd/api/v1/config/region-label/rules" RegionLabelRulesByIDs = "/pd/api/v1/config/region-label/rules/ids" @@ -79,13 +83,11 @@ func RegionByKey(key []byte) string { return fmt.Sprintf("%s/%s", regionByKey, url.QueryEscape(string(key))) } -// RegionsByKey returns the path of PD HTTP API to scan regions with given start key, end key and limit parameters. -func RegionsByKey(startKey, endKey []byte, limit int) string { +// RegionsByKeyRange returns the path of PD HTTP API to scan regions with given start key, end key and limit parameters. +func RegionsByKeyRange(keyRange *KeyRange, limit int) string { + startKeyStr, endKeyStr := keyRange.EscapeAsUTF8Str() return fmt.Sprintf("%s?start_key=%s&end_key=%s&limit=%d", - regionsByKey, - url.QueryEscape(string(startKey)), - url.QueryEscape(string(endKey)), - limit) + regionsByKey, startKeyStr, endKeyStr, limit) } // RegionsByStoreID returns the path of PD HTTP API to get regions by store ID. @@ -94,11 +96,10 @@ func RegionsByStoreID(storeID uint64) string { } // RegionStatsByKeyRange returns the path of PD HTTP API to get region stats by start key and end key. -func RegionStatsByKeyRange(startKey, endKey []byte) string { +func RegionStatsByKeyRange(keyRange *KeyRange) string { + startKeyStr, endKeyStr := keyRange.EscapeAsUTF8Str() return fmt.Sprintf("%s?start_key=%s&end_key=%s", - StatsRegion, - url.QueryEscape(string(startKey)), - url.QueryEscape(string(endKey))) + StatsRegion, startKeyStr, endKeyStr) } // StoreByID returns the store API with store ID parameter. @@ -136,6 +137,11 @@ func PlacementRuleBundleWithPartialParameter(partial bool) string { return fmt.Sprintf("%s?partial=%t", PlacementRuleBundle, partial) } +// PlacementRuleGroupByID returns the path of PD HTTP API to get placement rule group by ID. +func PlacementRuleGroupByID(id string) string { + return fmt.Sprintf("%s/%s", placementRuleGroup, id) +} + // SchedulerByName returns the scheduler API with the given scheduler name. func SchedulerByName(name string) string { return fmt.Sprintf("%s/%s", Schedulers, name) diff --git a/client/http/client.go b/client/http/client.go index 880489aa85c..d15693e11d4 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -22,7 +22,6 @@ import ( "fmt" "io" "net/http" - "net/url" "strings" "time" @@ -46,25 +45,35 @@ type Client interface { GetRegionByID(context.Context, uint64) (*RegionInfo, error) GetRegionByKey(context.Context, []byte) (*RegionInfo, error) GetRegions(context.Context) (*RegionsInfo, error) - GetRegionsByKeyRange(context.Context, []byte, []byte, int) (*RegionsInfo, error) + GetRegionsByKeyRange(context.Context, *KeyRange, int) (*RegionsInfo, error) GetRegionsByStoreID(context.Context, uint64) (*RegionsInfo, error) GetHotReadRegions(context.Context) (*StoreHotPeersInfos, error) GetHotWriteRegions(context.Context) (*StoreHotPeersInfos, error) - GetRegionStatusByKeyRange(context.Context, []byte, []byte) (*RegionStats, error) + GetHistoryHotRegions(context.Context, *HistoryHotRegionsRequest) (*HistoryHotRegions, error) + GetRegionStatusByKeyRange(context.Context, *KeyRange) (*RegionStats, error) GetStores(context.Context) (*StoresInfo, error) + /* Config-related interfaces */ + GetScheduleConfig(context.Context) (map[string]interface{}, error) + SetScheduleConfig(context.Context, map[string]interface{}) error /* Rule-related interfaces */ GetAllPlacementRuleBundles(context.Context) ([]*GroupBundle, error) GetPlacementRuleBundleByGroup(context.Context, string) (*GroupBundle, error) GetPlacementRulesByGroup(context.Context, string) ([]*Rule, error) SetPlacementRule(context.Context, *Rule) error + SetPlacementRuleInBatch(context.Context, []*RuleOp) error SetPlacementRuleBundles(context.Context, []*GroupBundle, bool) error DeletePlacementRule(context.Context, string, string) error + GetAllPlacementRuleGroups(context.Context) ([]*RuleGroup, error) + GetPlacementRuleGroupByID(context.Context, string) (*RuleGroup, error) + SetPlacementRuleGroup(context.Context, *RuleGroup) error + DeletePlacementRuleGroupByID(context.Context, string) error GetAllRegionLabelRules(context.Context) ([]*LabelRule, error) GetRegionLabelRulesByIDs(context.Context, []string) ([]*LabelRule, error) SetRegionLabelRule(context.Context, *LabelRule) error PatchRegionLabelRules(context.Context, *LabelRulePatch) error /* Scheduling-related interfaces */ - AccelerateSchedule(context.Context, []byte, []byte) error + AccelerateSchedule(context.Context, *KeyRange) error + AccelerateScheduleInBatch(context.Context, []*KeyRange) error /* Other interfaces */ GetMinResolvedTSByStoresIDs(context.Context, []uint64) (uint64, map[uint64]uint64, error) @@ -186,12 +195,23 @@ func (c *client) execDuration(name string, duration time.Duration) { c.executionDuration.WithLabelValues(name).Observe(duration.Seconds()) } +// HeaderOption configures the HTTP header. +type HeaderOption func(header http.Header) + +// WithAllowFollowerHandle sets the header field to allow a PD follower to handle this request. +func WithAllowFollowerHandle() HeaderOption { + return func(header http.Header) { + header.Set("PD-Allow-Follower-Handle", "true") + } +} + // At present, we will use the retry strategy of polling by default to keep // it consistent with the current implementation of some clients (e.g. TiDB). func (c *client) requestWithRetry( ctx context.Context, name, uri, method string, body io.Reader, res interface{}, + headerOpts ...HeaderOption, ) error { var ( err error @@ -199,7 +219,7 @@ func (c *client) requestWithRetry( ) for idx := 0; idx < len(c.pdAddrs); idx++ { addr = c.pdAddrs[idx] - err = c.request(ctx, name, fmt.Sprintf("%s%s", addr, uri), method, body, res) + err = c.request(ctx, name, fmt.Sprintf("%s%s", addr, uri), method, body, res, headerOpts...) if err == nil { break } @@ -213,6 +233,7 @@ func (c *client) request( ctx context.Context, name, url, method string, body io.Reader, res interface{}, + headerOpts ...HeaderOption, ) error { logFields := []zap.Field{ zap.String("name", name), @@ -224,6 +245,9 @@ func (c *client) request( log.Error("[pd] create http request failed", append(logFields, zap.Error(err))...) return errors.Trace(err) } + for _, opt := range headerOpts { + opt(req.Header) + } start := time.Now() resp, err := c.cli.Do(req) if err != nil { @@ -308,10 +332,11 @@ func (c *client) GetRegions(ctx context.Context) (*RegionsInfo, error) { } // GetRegionsByKeyRange gets the regions info by key range. If the limit is -1, it will return all regions within the range. -func (c *client) GetRegionsByKeyRange(ctx context.Context, startKey, endKey []byte, limit int) (*RegionsInfo, error) { +// The keys in the key range should be encoded in the UTF-8 bytes format. +func (c *client) GetRegionsByKeyRange(ctx context.Context, keyRange *KeyRange, limit int) (*RegionsInfo, error) { var regions RegionsInfo err := c.requestWithRetry(ctx, - "GetRegionsByKeyRange", RegionsByKey(startKey, endKey, limit), + "GetRegionsByKeyRange", RegionsByKeyRange(keyRange, limit), http.MethodGet, http.NoBody, ®ions) if err != nil { return nil, err @@ -355,11 +380,29 @@ func (c *client) GetHotWriteRegions(ctx context.Context) (*StoreHotPeersInfos, e return &hotWriteRegions, nil } +// GetHistoryHotRegions gets the history hot region statistics info. +func (c *client) GetHistoryHotRegions(ctx context.Context, req *HistoryHotRegionsRequest) (*HistoryHotRegions, error) { + reqJSON, err := json.Marshal(req) + if err != nil { + return nil, errors.Trace(err) + } + var historyHotRegions HistoryHotRegions + err = c.requestWithRetry(ctx, + "GetHistoryHotRegions", HotHistory, + http.MethodGet, bytes.NewBuffer(reqJSON), &historyHotRegions, + WithAllowFollowerHandle()) + if err != nil { + return nil, err + } + return &historyHotRegions, nil +} + // GetRegionStatusByKeyRange gets the region status by key range. -func (c *client) GetRegionStatusByKeyRange(ctx context.Context, startKey, endKey []byte) (*RegionStats, error) { +// The keys in the key range should be encoded in the UTF-8 bytes format. +func (c *client) GetRegionStatusByKeyRange(ctx context.Context, keyRange *KeyRange) (*RegionStats, error) { var regionStats RegionStats err := c.requestWithRetry(ctx, - "GetRegionStatusByKeyRange", RegionStatsByKeyRange(startKey, endKey), + "GetRegionStatusByKeyRange", RegionStatsByKeyRange(keyRange), http.MethodGet, http.NoBody, ®ionStats, ) if err != nil { @@ -368,6 +411,29 @@ func (c *client) GetRegionStatusByKeyRange(ctx context.Context, startKey, endKey return ®ionStats, nil } +// GetScheduleConfig gets the schedule configurations. +func (c *client) GetScheduleConfig(ctx context.Context) (map[string]interface{}, error) { + var config map[string]interface{} + err := c.requestWithRetry(ctx, + "GetScheduleConfig", ScheduleConfig, + http.MethodGet, http.NoBody, &config) + if err != nil { + return nil, err + } + return config, nil +} + +// SetScheduleConfig sets the schedule configurations. +func (c *client) SetScheduleConfig(ctx context.Context, config map[string]interface{}) error { + configJSON, err := json.Marshal(config) + if err != nil { + return errors.Trace(err) + } + return c.requestWithRetry(ctx, + "SetScheduleConfig", ScheduleConfig, + http.MethodPost, bytes.NewBuffer(configJSON), nil) +} + // GetStores gets the stores info. func (c *client) GetStores(ctx context.Context) (*StoresInfo, error) { var stores StoresInfo @@ -427,6 +493,17 @@ func (c *client) SetPlacementRule(ctx context.Context, rule *Rule) error { http.MethodPost, bytes.NewBuffer(ruleJSON), nil) } +// SetPlacementRuleInBatch sets the placement rules in batch. +func (c *client) SetPlacementRuleInBatch(ctx context.Context, ruleOps []*RuleOp) error { + ruleOpsJSON, err := json.Marshal(ruleOps) + if err != nil { + return errors.Trace(err) + } + return c.requestWithRetry(ctx, + "SetPlacementRuleInBatch", PlacementRulesInBatch, + http.MethodPost, bytes.NewBuffer(ruleOpsJSON), nil) +} + // SetPlacementRuleBundles sets the placement rule bundles. // If `partial` is false, all old configurations will be over-written and dropped. func (c *client) SetPlacementRuleBundles(ctx context.Context, bundles []*GroupBundle, partial bool) error { @@ -446,6 +523,48 @@ func (c *client) DeletePlacementRule(ctx context.Context, group, id string) erro http.MethodDelete, http.NoBody, nil) } +// GetAllPlacementRuleGroups gets all placement rule groups. +func (c *client) GetAllPlacementRuleGroups(ctx context.Context) ([]*RuleGroup, error) { + var ruleGroups []*RuleGroup + err := c.requestWithRetry(ctx, + "GetAllPlacementRuleGroups", placementRuleGroups, + http.MethodGet, http.NoBody, &ruleGroups) + if err != nil { + return nil, err + } + return ruleGroups, nil +} + +// GetPlacementRuleGroupByID gets the placement rule group by ID. +func (c *client) GetPlacementRuleGroupByID(ctx context.Context, id string) (*RuleGroup, error) { + var ruleGroup RuleGroup + err := c.requestWithRetry(ctx, + "GetPlacementRuleGroupByID", PlacementRuleGroupByID(id), + http.MethodGet, http.NoBody, &ruleGroup) + if err != nil { + return nil, err + } + return &ruleGroup, nil +} + +// SetPlacementRuleGroup sets the placement rule group. +func (c *client) SetPlacementRuleGroup(ctx context.Context, ruleGroup *RuleGroup) error { + ruleGroupJSON, err := json.Marshal(ruleGroup) + if err != nil { + return errors.Trace(err) + } + return c.requestWithRetry(ctx, + "SetPlacementRuleGroup", placementRuleGroup, + http.MethodPost, bytes.NewBuffer(ruleGroupJSON), nil) +} + +// DeletePlacementRuleGroupByID deletes the placement rule group by ID. +func (c *client) DeletePlacementRuleGroupByID(ctx context.Context, id string) error { + return c.requestWithRetry(ctx, + "DeletePlacementRuleGroupByID", PlacementRuleGroupByID(id), + http.MethodDelete, http.NoBody, nil) +} + // GetAllRegionLabelRules gets all region label rules. func (c *client) GetAllRegionLabelRules(ctx context.Context) ([]*LabelRule, error) { var labelRules []*LabelRule @@ -497,17 +616,38 @@ func (c *client) PatchRegionLabelRules(ctx context.Context, labelRulePatch *Labe } // AccelerateSchedule accelerates the scheduling of the regions within the given key range. -func (c *client) AccelerateSchedule(ctx context.Context, startKey, endKey []byte) error { - input := map[string]string{ - "start_key": url.QueryEscape(string(startKey)), - "end_key": url.QueryEscape(string(endKey)), +// The keys in the key range should be encoded in the hex bytes format (without encoding to the UTF-8 bytes). +func (c *client) AccelerateSchedule(ctx context.Context, keyRange *KeyRange) error { + startKey, endKey := keyRange.EscapeAsHexStr() + inputJSON, err := json.Marshal(map[string]string{ + "start_key": startKey, + "end_key": endKey, + }) + if err != nil { + return errors.Trace(err) + } + return c.requestWithRetry(ctx, + "AccelerateSchedule", AccelerateSchedule, + http.MethodPost, bytes.NewBuffer(inputJSON), nil) +} + +// AccelerateScheduleInBatch accelerates the scheduling of the regions within the given key ranges in batch. +// The keys in the key ranges should be encoded in the hex bytes format (without encoding to the UTF-8 bytes). +func (c *client) AccelerateScheduleInBatch(ctx context.Context, keyRanges []*KeyRange) error { + input := make([]map[string]string, 0, len(keyRanges)) + for _, keyRange := range keyRanges { + startKey, endKey := keyRange.EscapeAsHexStr() + input = append(input, map[string]string{ + "start_key": startKey, + "end_key": endKey, + }) } inputJSON, err := json.Marshal(input) if err != nil { return errors.Trace(err) } return c.requestWithRetry(ctx, - "AccelerateSchedule", AccelerateSchedule, + "AccelerateScheduleInBatch", AccelerateScheduleInBatch, http.MethodPost, bytes.NewBuffer(inputJSON), nil) } diff --git a/client/http/types.go b/client/http/types.go index f948286c2b5..4e99d911e0b 100644 --- a/client/http/types.go +++ b/client/http/types.go @@ -14,7 +14,46 @@ package http -import "time" +import ( + "encoding/hex" + "encoding/json" + "net/url" + "time" + + "github.com/pingcap/kvproto/pkg/encryptionpb" +) + +// KeyRange defines a range of keys in bytes. +type KeyRange struct { + startKey []byte + endKey []byte +} + +// NewKeyRange creates a new key range structure with the given start key and end key bytes. +// Notice: the actual encoding of the key range is not specified here. It should be either UTF-8 or hex. +// - UTF-8 means the key has already been encoded into a string with UTF-8 encoding, like: +// []byte{52 56 54 53 54 99 54 99 54 102 50 48 53 55 54 102 55 50 54 99 54 52}, which will later be converted to "48656c6c6f20576f726c64" +// by using `string()` method. +// - Hex means the key is just a raw hex bytes without encoding to a UTF-8 string, like: +// []byte{72, 101, 108, 108, 111, 32, 87, 111, 114, 108, 100}, which will later be converted to "48656c6c6f20576f726c64" +// by using `hex.EncodeToString()` method. +func NewKeyRange(startKey, endKey []byte) *KeyRange { + return &KeyRange{startKey, endKey} +} + +// EscapeAsUTF8Str returns the URL escaped key strings as they are UTF-8 encoded. +func (r *KeyRange) EscapeAsUTF8Str() (startKeyStr, endKeyStr string) { + startKeyStr = url.QueryEscape(string(r.startKey)) + endKeyStr = url.QueryEscape(string(r.endKey)) + return +} + +// EscapeAsHexStr returns the URL escaped key strings as they are hex encoded. +func (r *KeyRange) EscapeAsHexStr() (startKeyStr, endKeyStr string) { + startKeyStr = url.QueryEscape(hex.EncodeToString(r.startKey)) + endKeyStr = url.QueryEscape(hex.EncodeToString(r.endKey)) + return +} // NOTICE: the structures below are copied from the PD API definitions. // Please make sure the consistency if any change happens to the PD API. @@ -129,6 +168,46 @@ type HotPeerStatShow struct { LastUpdateTime time.Time `json:"last_update_time,omitempty"` } +// HistoryHotRegionsRequest wrap the request conditions. +type HistoryHotRegionsRequest struct { + StartTime int64 `json:"start_time,omitempty"` + EndTime int64 `json:"end_time,omitempty"` + RegionIDs []uint64 `json:"region_ids,omitempty"` + StoreIDs []uint64 `json:"store_ids,omitempty"` + PeerIDs []uint64 `json:"peer_ids,omitempty"` + IsLearners []bool `json:"is_learners,omitempty"` + IsLeaders []bool `json:"is_leaders,omitempty"` + HotRegionTypes []string `json:"hot_region_type,omitempty"` +} + +// HistoryHotRegions wraps historyHotRegion +type HistoryHotRegions struct { + HistoryHotRegion []*HistoryHotRegion `json:"history_hot_region"` +} + +// HistoryHotRegion wraps hot region info +// it is storage format of hot_region_storage +type HistoryHotRegion struct { + UpdateTime int64 `json:"update_time"` + RegionID uint64 `json:"region_id"` + PeerID uint64 `json:"peer_id"` + StoreID uint64 `json:"store_id"` + IsLeader bool `json:"is_leader"` + IsLearner bool `json:"is_learner"` + HotRegionType string `json:"hot_region_type"` + HotDegree int64 `json:"hot_degree"` + FlowBytes float64 `json:"flow_bytes"` + KeyRate float64 `json:"key_rate"` + QueryRate float64 `json:"query_rate"` + StartKey string `json:"start_key"` + EndKey string `json:"end_key"` + // Encryption metadata for start_key and end_key. encryption_meta.iv is IV for start_key. + // IV for end_key is calculated from (encryption_meta.iv + len(start_key)). + // The field is only used by PD and should be ignored otherwise. + // If encryption_meta is empty (i.e. nil), it means start_key and end_key are unencrypted. + EncryptionMeta *encryptionpb.EncryptionMeta `json:"encryption_meta,omitempty"` +} + // StoresInfo represents the information of all TiKV/TiFlash stores. type StoresInfo struct { Count int `json:"count"` @@ -247,6 +326,56 @@ type Rule struct { CreateTimestamp uint64 `json:"create_timestamp,omitempty"` // only set at runtime, recorded rule create timestamp } +// String returns the string representation of this rule. +func (r *Rule) String() string { + b, _ := json.Marshal(r) + return string(b) +} + +// Clone returns a copy of Rule. +func (r *Rule) Clone() *Rule { + var clone Rule + json.Unmarshal([]byte(r.String()), &clone) + clone.StartKey = append(r.StartKey[:0:0], r.StartKey...) + clone.EndKey = append(r.EndKey[:0:0], r.EndKey...) + return &clone +} + +// RuleOpType indicates the operation type +type RuleOpType string + +const ( + // RuleOpAdd a placement rule, only need to specify the field *Rule + RuleOpAdd RuleOpType = "add" + // RuleOpDel a placement rule, only need to specify the field `GroupID`, `ID`, `MatchID` + RuleOpDel RuleOpType = "del" +) + +// RuleOp is for batching placement rule actions. +// The action type is distinguished by the field `Action`. +type RuleOp struct { + *Rule // information of the placement rule to add/delete the operation type + Action RuleOpType `json:"action"` + DeleteByIDPrefix bool `json:"delete_by_id_prefix"` // if action == delete, delete by the prefix of id +} + +func (r RuleOp) String() string { + b, _ := json.Marshal(r) + return string(b) +} + +// RuleGroup defines properties of a rule group. +type RuleGroup struct { + ID string `json:"id,omitempty"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` +} + +func (g *RuleGroup) String() string { + b, _ := json.Marshal(g) + return string(b) +} + // GroupBundle represents a rule group and all rules belong to the group. type GroupBundle struct { ID string `json:"group_id"` diff --git a/client/option.go b/client/option.go index d6a6d61d2f9..2a6c285cfb7 100644 --- a/client/option.go +++ b/client/option.go @@ -28,6 +28,7 @@ const ( maxInitClusterRetries = 100 defaultMaxTSOBatchWaitInterval time.Duration = 0 defaultEnableTSOFollowerProxy = false + defaultEnableFollowerHandle = false ) // DynamicOption is used to distinguish the dynamic option type. @@ -40,6 +41,8 @@ const ( // EnableTSOFollowerProxy is the TSO Follower Proxy option. // It is stored as bool. EnableTSOFollowerProxy + // EnableFollowerHandle is the follower handle option. + EnableFollowerHandle dynamicOptionCount ) @@ -72,6 +75,7 @@ func newOption() *option { co.dynamicOptions[MaxTSOBatchWaitInterval].Store(defaultMaxTSOBatchWaitInterval) co.dynamicOptions[EnableTSOFollowerProxy].Store(defaultEnableTSOFollowerProxy) + co.dynamicOptions[EnableFollowerHandle].Store(defaultEnableFollowerHandle) return co } @@ -88,6 +92,19 @@ func (o *option) setMaxTSOBatchWaitInterval(interval time.Duration) error { return nil } +// setEnableFollowerHandle set the Follower Handle option. +func (o *option) setEnableFollowerHandle(enable bool) { + old := o.getEnableFollowerHandle() + if enable != old { + o.dynamicOptions[EnableFollowerHandle].Store(enable) + } +} + +// getMaxTSOBatchWaitInterval gets the Follower Handle enable option. +func (o *option) getEnableFollowerHandle() bool { + return o.dynamicOptions[EnableFollowerHandle].Load().(bool) +} + // getMaxTSOBatchWaitInterval gets the max TSO batch wait interval option. func (o *option) getMaxTSOBatchWaitInterval() time.Duration { return o.dynamicOptions[MaxTSOBatchWaitInterval].Load().(time.Duration) diff --git a/client/option_test.go b/client/option_test.go index 1b5604f4d19..1a8faf8fcd9 100644 --- a/client/option_test.go +++ b/client/option_test.go @@ -28,6 +28,7 @@ func TestDynamicOptionChange(t *testing.T) { // Check the default value setting. re.Equal(defaultMaxTSOBatchWaitInterval, o.getMaxTSOBatchWaitInterval()) re.Equal(defaultEnableTSOFollowerProxy, o.getEnableTSOFollowerProxy()) + re.Equal(defaultEnableFollowerHandle, o.getEnableFollowerHandle()) // Check the invalid value setting. re.NotNil(o.setMaxTSOBatchWaitInterval(time.Second)) @@ -55,4 +56,11 @@ func TestDynamicOptionChange(t *testing.T) { close(o.enableTSOFollowerProxyCh) // Setting the same value should not notify the channel. o.setEnableTSOFollowerProxy(expectBool) + + expectBool = true + o.setEnableFollowerHandle(expectBool) + re.Equal(expectBool, o.getEnableFollowerHandle()) + expectBool = false + o.setEnableFollowerHandle(expectBool) + re.Equal(expectBool, o.getEnableFollowerHandle()) } diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index e7a2e16fdd9..b75276adbe9 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -63,8 +63,6 @@ type ServiceDiscovery interface { GetKeyspaceID() uint32 // GetKeyspaceGroupID returns the ID of the keyspace group GetKeyspaceGroupID() uint32 - // DiscoverServiceURLs discovers the microservice with the specified type and returns the server urls. - DiscoverMicroservice(svcType serviceType) ([]string, error) // GetServiceURLs returns the URLs of the servers providing the service GetServiceURLs() []string // GetServingEndpointClientConn returns the grpc client connection of the serving endpoint @@ -324,7 +322,7 @@ func (c *pdServiceDiscovery) GetKeyspaceGroupID() uint32 { } // 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 *pdServiceDiscovery) discoverMicroservice(svcType serviceType) (urls []string, err error) { switch svcType { case apiService: urls = c.GetServiceURLs() diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index 92f95129951..5f14c406797 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -288,21 +288,6 @@ func (c *tsoServiceDiscovery) GetKeyspaceGroupID() uint32 { return c.keyspaceGroupSD.group.Id } -// DiscoverServiceURLs discovers the microservice with the specified type and returns the server urls. -func (c *tsoServiceDiscovery) DiscoverMicroservice(svcType serviceType) ([]string, error) { - var urls []string - - switch svcType { - case apiService: - case tsoService: - return c.apiSvcDiscovery.DiscoverMicroservice(tsoService) - default: - panic("invalid service type") - } - - return urls, nil -} - // GetServiceURLs returns the URLs of the tso primary/secondary addresses of this keyspace group. // For testing use. It should only be called when the client is closed. func (c *tsoServiceDiscovery) GetServiceURLs() []string { @@ -582,7 +567,7 @@ func (c *tsoServiceDiscovery) getTSOServer(sd ServiceDiscovery) (string, error) ) t := c.tsoServerDiscovery if len(t.addrs) == 0 || t.failureCount == len(t.addrs) { - addrs, err = sd.DiscoverMicroservice(tsoService) + addrs, err = sd.(*pdServiceDiscovery).discoverMicroservice(tsoService) if err != nil { return "", err } diff --git a/go.mod b/go.mod index 0306d70f7a3..676d350d22d 100644 --- a/go.mod +++ b/go.mod @@ -18,7 +18,7 @@ require ( github.com/gin-contrib/cors v1.4.0 github.com/gin-contrib/gzip v0.0.1 github.com/gin-contrib/pprof v1.4.0 - github.com/gin-gonic/gin v1.8.1 + github.com/gin-gonic/gin v1.9.1 github.com/go-echarts/go-echarts v1.0.0 github.com/gogo/protobuf v1.3.2 github.com/google/btree v1.1.2 @@ -36,7 +36,7 @@ require ( github.com/pingcap/kvproto v0.0.0-20231018065736-c0689aded40c 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-20231108071238-7cb8b7ff0537 + github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e github.com/prometheus/client_golang v1.11.1 github.com/prometheus/common v0.26.0 github.com/sasha-s/go-deadlock v0.2.0 @@ -45,7 +45,7 @@ require ( github.com/soheilhy/cmux v0.1.4 github.com/spf13/cobra v1.0.0 github.com/spf13/pflag v1.0.5 - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.3 github.com/swaggo/http-swagger v1.2.6 github.com/swaggo/swag v1.8.3 github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 @@ -84,8 +84,10 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect + github.com/bytedance/sonic v1.9.1 // indirect github.com/cenkalti/backoff/v4 v4.0.2 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect github.com/cpuguy83/go-md2man/v2 v2.0.0 // indirect @@ -96,6 +98,7 @@ require ( github.com/fatih/structtag v1.2.0 // indirect github.com/fogleman/gg v1.3.0 // indirect github.com/fsnotify/fsnotify v1.4.9 // indirect + github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/ghodss/yaml v1.0.0 // indirect github.com/gin-contrib/sse v0.1.0 // indirect github.com/go-ole/go-ole v1.2.6 // indirect @@ -103,13 +106,13 @@ require ( github.com/go-openapi/jsonreference v0.19.6 // indirect github.com/go-openapi/spec v0.20.4 // indirect github.com/go-openapi/swag v0.19.15 // indirect - github.com/go-playground/locales v0.14.0 // indirect - github.com/go-playground/universal-translator v0.18.0 // indirect - github.com/go-playground/validator/v10 v10.10.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.0.9 // indirect - github.com/goccy/go-json v0.9.7 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect github.com/golang/protobuf v1.5.3 // indirect @@ -128,11 +131,12 @@ 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/cpuid/v2 v2.2.4 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/leodido/go-urn v1.2.1 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect github.com/mattn/go-colorable v0.1.8 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/mattn/go-runewidth v0.0.8 // indirect github.com/mattn/go-sqlite3 v1.14.15 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect @@ -144,7 +148,7 @@ require ( github.com/oleiade/reflections v1.0.1 // indirect github.com/olekukonko/tablewriter v0.0.4 // indirect github.com/onsi/gomega v1.20.1 // indirect - github.com/pelletier/go-toml/v2 v2.0.1 // indirect + github.com/pelletier/go-toml/v2 v2.0.8 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect @@ -166,7 +170,8 @@ require ( github.com/tklauser/go-sysconf v0.3.11 // indirect github.com/tklauser/numcpus v0.6.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 // indirect - github.com/ugorji/go/codec v1.2.7 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/ugorji/go/codec v1.2.11 // indirect github.com/urfave/cli/v2 v2.3.0 // indirect github.com/vmihailenco/msgpack/v5 v5.3.5 // indirect github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect @@ -177,6 +182,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect + golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.14.0 // indirect golang.org/x/image v0.5.0 // indirect golang.org/x/mod v0.8.0 // indirect diff --git a/go.sum b/go.sum index fb178321864..c7ceeee028c 100644 --- a/go.sum +++ b/go.sum @@ -72,6 +72,9 @@ github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4Yn github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpfpgJhvkuYWBmOYbo= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.0.2 h1:JIufpQLbh4DkbQoii76ItQIUFzevQSqOLZca4eamEDs= @@ -81,6 +84,9 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= 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= @@ -140,6 +146,8 @@ github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzP github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= +github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/cors v1.4.0 h1:oJ6gwtUl3lqV0WEIwM/LxPF1QZ5qe2lGWdY2+bz7y0g= @@ -153,8 +161,9 @@ github.com/gin-contrib/sse v0.1.0 h1:Y/yl/+YNO8GZSjAhjMsSuLt29uWRFHdHYUb5lYOV9qE github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= -github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= +github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= +github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= github.com/go-echarts/go-echarts v1.0.0 h1:n181E4iXwj4zrU9VYmdM2m8dyhERt2w9k9YhHqdp6A8= github.com/go-echarts/go-echarts v1.0.0/go.mod h1:qbmyAb/Rl1f2w7wKba1D4LoNq4U164yO4/wedFbcWyo= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= @@ -176,17 +185,21 @@ github.com/go-openapi/spec v0.20.4/go.mod h1:faYFR1CvsJZ0mNsmsphTMSoRrNV3TEDoAM7 github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.15 h1:D2NRCBzS9/pEY3gP9Nl8aDqGUcPFrwG2p+CNFrLyrCM= github.com/go-openapi/swag v0.19.15/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= -github.com/go-playground/assert/v2 v2.0.1 h1:MsBgLAaY856+nPRTKrp3/OZK38U/wa0CcBYNjji3q3A= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/locales v0.14.0 h1:u50s323jtVGugKlcYeyzC0etD1HifMjqmJqb8WugfUU= github.com/go-playground/locales v0.14.0/go.mod h1:sawfccIbzZTqEDETgFXqTho0QybSa7l++s0DH+LDiLs= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= -github.com/go-playground/universal-translator v0.18.0 h1:82dyy6p4OuJq4/CByFNOn/jYrnRPArHwAcmLoJZxyho= github.com/go-playground/universal-translator v0.18.0/go.mod h1:UvRDBj+xPUEGrFYl+lu/H90nyDXpg0fqeB/AQUGNTVA= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= -github.com/go-playground/validator/v10 v10.10.0 h1:I7mrTYv78z8k8VXa/qJlOlEXn/nBh+BF8dHX5nt/dr0= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= +github.com/go-playground/validator/v10 v10.14.0 h1:vgvQWe3XCz3gIeFDm/HnTIbj6UGmg/+t63MyGU2n5js= +github.com/go-playground/validator/v10 v10.14.0/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -194,8 +207,9 @@ github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9 github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-graphviz v0.0.9 h1:s/FMMJ1Joj6La3S5ApO3Jk2cwM4LpXECC2muFx3IPQQ= github.com/goccy/go-graphviz v0.0.9/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= -github.com/goccy/go-json v0.9.7 h1:IcB+Aqpx/iMHu5Yooh7jEzJk1JZ7Pjtmys2ukPr7EeM= github.com/goccy/go-json v0.9.7/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -335,6 +349,9 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= 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/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -348,8 +365,9 @@ 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/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= -github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= @@ -366,8 +384,9 @@ github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNx github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.8 h1:3tS41NlGYSmhhe/8fhGRzc+z3AYCw1Fe1WAyLuujKs0= @@ -421,8 +440,9 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml/v2 v2.0.1 h1:8e3L2cCQzLFi2CR4g7vGFuFxX7Jl1kKX8gW+iV0GUKU= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= +github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -446,8 +466,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 h1:wnHt7ETIB0vm+gbLx8QhcIEmRtrT4QlWlfpcI9vjxOk= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537/go.mod h1:EZ90+V5S4TttbYag6oKZ3jcNKRwZe1Mc9vXwOt9JBYw= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e h1:SJUSDejvKtj9vSh5ptRHh4iMrvPV3oKO8yp6/SYE8vc= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -548,8 +568,10 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ 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.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -577,13 +599,16 @@ github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go v1.2.7/go.mod h1:nF9osbDWLy6bDVv/Rtoh6QgnvNDpmCalQV5urGCCS6M= github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= +github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= +github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= @@ -642,6 +667,9 @@ go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -756,6 +784,7 @@ golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/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-20220704084225-05e143d24a9e/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.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -900,6 +929,7 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= moul.io/zapgorm2 v1.1.0 h1:qwAlMBYf+qJkJ7PAzJl4oCe6eS6QGiKAXUPeis0+RBE= moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/pkg/core/store_option.go b/pkg/core/store_option.go index 0bdfaffa44f..93b25562731 100644 --- a/pkg/core/store_option.go +++ b/pkg/core/store_option.go @@ -276,6 +276,7 @@ func SetLastAwakenTime(lastAwaken time.Time) StoreCreateOption { } // SetStoreMeta sets the meta for the store. +// NOTICE: LastHeartbeat is not persisted each time, so it is not set by this function. Please use SetLastHeartbeatTS instead. func SetStoreMeta(newMeta *metapb.Store) StoreCreateOption { return func(store *StoreInfo) { meta := typeutil.DeepClone(store.meta, StoreFactory) @@ -286,7 +287,6 @@ func SetStoreMeta(newMeta *metapb.Store) StoreCreateOption { meta.PeerAddress = newMeta.GetPeerAddress() meta.StartTimestamp = newMeta.GetStartTimestamp() meta.DeployPath = newMeta.GetDeployPath() - meta.LastHeartbeat = newMeta.GetLastHeartbeat() meta.State = newMeta.GetState() meta.Labels = newMeta.GetLabels() meta.NodeState = newMeta.GetNodeState() diff --git a/pkg/dashboard/adapter/manager.go b/pkg/dashboard/adapter/manager.go index a3691242c8f..293d8ad6549 100644 --- a/pkg/dashboard/adapter/manager.go +++ b/pkg/dashboard/adapter/manager.go @@ -19,6 +19,7 @@ import ( "sync" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" "github.com/pingcap/tidb-dashboard/pkg/apiserver" @@ -75,6 +76,10 @@ func (m *Manager) Stop() { func (m *Manager) serviceLoop() { defer logutil.LogPanic() defer m.wg.Done() + // TODO: After we fix the atomic problem of config, we can remove this failpoint. + failpoint.Inject("skipDashboardLoop", func() { + failpoint.Return() + }) ticker := time.NewTicker(CheckInterval) defer ticker.Stop() diff --git a/pkg/mcs/scheduling/server/cluster.go b/pkg/mcs/scheduling/server/cluster.go index 59b2e691658..b24db7ac805 100644 --- a/pkg/mcs/scheduling/server/cluster.go +++ b/pkg/mcs/scheduling/server/cluster.go @@ -381,7 +381,8 @@ func (c *Cluster) HandleStoreHeartbeat(heartbeat *schedulingpb.StoreHeartbeatReq return errors.Errorf("store %v not found", storeID) } - newStore := store.Clone(core.SetStoreStats(stats)) + nowTime := time.Now() + newStore := store.Clone(core.SetStoreStats(stats), core.SetLastHeartbeatTS(nowTime)) if store := c.GetStore(storeID); store != nil { statistics.UpdateStoreHeartbeatMetrics(store) diff --git a/pkg/mcs/scheduling/server/config/config.go b/pkg/mcs/scheduling/server/config/config.go index a211c989c64..6c38f66dc68 100644 --- a/pkg/mcs/scheduling/server/config/config.go +++ b/pkg/mcs/scheduling/server/config/config.go @@ -19,6 +19,7 @@ import ( "os" "path/filepath" "reflect" + "strconv" "strings" "sync/atomic" "time" @@ -30,6 +31,7 @@ import ( "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" "github.com/tikv/pd/pkg/mcs/utils" @@ -220,6 +222,7 @@ func (c *Config) Clone() *Config { // PersistConfig wraps all configurations that need to persist to storage and // allows to access them safely. type PersistConfig struct { + ttl *cache.TTLString // Store the global configuration that is related to the scheduling. clusterVersion unsafe.Pointer schedule atomic.Value @@ -231,7 +234,7 @@ type PersistConfig struct { } // NewPersistConfig creates a new PersistConfig instance. -func NewPersistConfig(cfg *Config) *PersistConfig { +func NewPersistConfig(cfg *Config, ttl *cache.TTLString) *PersistConfig { o := &PersistConfig{} o.SetClusterVersion(&cfg.ClusterVersion) o.schedule.Store(&cfg.Schedule) @@ -239,6 +242,7 @@ func NewPersistConfig(cfg *Config) *PersistConfig { // storeConfig will be fetched from TiKV by PD API server, // so we just set an empty value here first. o.storeConfig.Store(&sc.StoreConfig{}) + o.ttl = ttl return o } @@ -329,16 +333,6 @@ func (o *PersistConfig) GetMaxReplicas() int { return int(o.GetReplicationConfig().MaxReplicas) } -// GetMaxSnapshotCount returns the max snapshot count. -func (o *PersistConfig) GetMaxSnapshotCount() uint64 { - return o.GetScheduleConfig().MaxSnapshotCount -} - -// GetMaxPendingPeerCount returns the max pending peer count. -func (o *PersistConfig) GetMaxPendingPeerCount() uint64 { - return o.GetScheduleConfig().MaxPendingPeerCount -} - // IsPlacementRulesEnabled returns if the placement rules is enabled. func (o *PersistConfig) IsPlacementRulesEnabled() bool { return o.GetReplicationConfig().EnablePlacementRules @@ -354,31 +348,6 @@ func (o *PersistConfig) GetHighSpaceRatio() float64 { return o.GetScheduleConfig().HighSpaceRatio } -// GetHotRegionScheduleLimit returns the limit for hot region schedule. -func (o *PersistConfig) GetHotRegionScheduleLimit() uint64 { - return o.GetScheduleConfig().HotRegionScheduleLimit -} - -// GetRegionScheduleLimit returns the limit for region schedule. -func (o *PersistConfig) GetRegionScheduleLimit() uint64 { - return o.GetScheduleConfig().RegionScheduleLimit -} - -// GetLeaderScheduleLimit returns the limit for leader schedule. -func (o *PersistConfig) GetLeaderScheduleLimit() uint64 { - return o.GetScheduleConfig().LeaderScheduleLimit -} - -// GetReplicaScheduleLimit returns the limit for replica schedule. -func (o *PersistConfig) GetReplicaScheduleLimit() uint64 { - return o.GetScheduleConfig().ReplicaScheduleLimit -} - -// GetMergeScheduleLimit returns the limit for merge schedule. -func (o *PersistConfig) GetMergeScheduleLimit() uint64 { - return o.GetScheduleConfig().MergeScheduleLimit -} - // GetLeaderSchedulePolicy is to get leader schedule policy. func (o *PersistConfig) GetLeaderSchedulePolicy() constant.SchedulePolicy { return constant.StringToSchedulePolicy(o.GetScheduleConfig().LeaderSchedulePolicy) @@ -419,26 +388,11 @@ func (o *PersistConfig) IsOneWayMergeEnabled() bool { return o.GetScheduleConfig().EnableOneWayMerge } -// GetMaxMergeRegionSize returns the max region size. -func (o *PersistConfig) GetMaxMergeRegionSize() uint64 { - return o.GetScheduleConfig().MaxMergeRegionSize -} - -// GetMaxMergeRegionKeys returns the max region keys. -func (o *PersistConfig) GetMaxMergeRegionKeys() uint64 { - return o.GetScheduleConfig().MaxMergeRegionKeys -} - // GetRegionScoreFormulaVersion returns the region score formula version. func (o *PersistConfig) GetRegionScoreFormulaVersion() string { return o.GetScheduleConfig().RegionScoreFormulaVersion } -// GetSchedulerMaxWaitingOperator returns the scheduler max waiting operator. -func (o *PersistConfig) GetSchedulerMaxWaitingOperator() uint64 { - return o.GetScheduleConfig().SchedulerMaxWaitingOperator -} - // GetHotRegionCacheHitsThreshold returns the hot region cache hits threshold. func (o *PersistConfig) GetHotRegionCacheHitsThreshold() int { return int(o.GetScheduleConfig().HotRegionCacheHitsThreshold) @@ -474,11 +428,6 @@ func (o *PersistConfig) GetTolerantSizeRatio() float64 { return o.GetScheduleConfig().TolerantSizeRatio } -// GetWitnessScheduleLimit returns the limit for region schedule. -func (o *PersistConfig) GetWitnessScheduleLimit() uint64 { - return o.GetScheduleConfig().WitnessScheduleLimit -} - // IsDebugMetricsEnabled returns if debug metrics is enabled. func (o *PersistConfig) IsDebugMetricsEnabled() bool { return o.GetScheduleConfig().EnableDebugMetrics @@ -509,11 +458,6 @@ func (o *PersistConfig) IsRemoveExtraReplicaEnabled() bool { return o.GetScheduleConfig().EnableRemoveExtraReplica } -// IsLocationReplacementEnabled returns if location replace is enabled. -func (o *PersistConfig) IsLocationReplacementEnabled() bool { - return o.GetScheduleConfig().EnableLocationReplacement -} - // IsWitnessAllowed returns if the witness is allowed. func (o *PersistConfig) IsWitnessAllowed() bool { return o.GetScheduleConfig().EnableWitness @@ -534,8 +478,87 @@ func (o *PersistConfig) GetStoresLimit() map[uint64]sc.StoreLimitConfig { return o.GetScheduleConfig().StoreLimit } +// TTL related methods. + +// GetLeaderScheduleLimit returns the limit for leader schedule. +func (o *PersistConfig) GetLeaderScheduleLimit() uint64 { + return o.getTTLUintOr(sc.LeaderScheduleLimitKey, o.GetScheduleConfig().LeaderScheduleLimit) +} + +// GetRegionScheduleLimit returns the limit for region schedule. +func (o *PersistConfig) GetRegionScheduleLimit() uint64 { + return o.getTTLUintOr(sc.RegionScheduleLimitKey, o.GetScheduleConfig().RegionScheduleLimit) +} + +// GetWitnessScheduleLimit returns the limit for region schedule. +func (o *PersistConfig) GetWitnessScheduleLimit() uint64 { + return o.getTTLUintOr(sc.WitnessScheduleLimitKey, o.GetScheduleConfig().WitnessScheduleLimit) +} + +// GetReplicaScheduleLimit returns the limit for replica schedule. +func (o *PersistConfig) GetReplicaScheduleLimit() uint64 { + return o.getTTLUintOr(sc.ReplicaRescheduleLimitKey, o.GetScheduleConfig().ReplicaScheduleLimit) +} + +// GetMergeScheduleLimit returns the limit for merge schedule. +func (o *PersistConfig) GetMergeScheduleLimit() uint64 { + return o.getTTLUintOr(sc.MergeScheduleLimitKey, o.GetScheduleConfig().MergeScheduleLimit) +} + +// GetHotRegionScheduleLimit returns the limit for hot region schedule. +func (o *PersistConfig) GetHotRegionScheduleLimit() uint64 { + return o.getTTLUintOr(sc.HotRegionScheduleLimitKey, o.GetScheduleConfig().HotRegionScheduleLimit) +} + +// GetStoreLimit returns the limit of a store. +func (o *PersistConfig) GetStoreLimit(storeID uint64) (returnSC sc.StoreLimitConfig) { + defer func() { + returnSC.RemovePeer = o.getTTLFloatOr(fmt.Sprintf("remove-peer-%v", storeID), returnSC.RemovePeer) + returnSC.AddPeer = o.getTTLFloatOr(fmt.Sprintf("add-peer-%v", storeID), returnSC.AddPeer) + }() + if limit, ok := o.GetScheduleConfig().StoreLimit[storeID]; ok { + return limit + } + cfg := o.GetScheduleConfig().Clone() + sc := sc.StoreLimitConfig{ + AddPeer: sc.DefaultStoreLimit.GetDefaultStoreLimit(storelimit.AddPeer), + RemovePeer: sc.DefaultStoreLimit.GetDefaultStoreLimit(storelimit.RemovePeer), + } + v, ok1, err := o.getTTLFloat("default-add-peer") + if err != nil { + log.Warn("failed to parse default-add-peer from PersistOptions's ttl storage", zap.Error(err)) + } + canSetAddPeer := ok1 && err == nil + if canSetAddPeer { + returnSC.AddPeer = v + } + + v, ok2, err := o.getTTLFloat("default-remove-peer") + if err != nil { + log.Warn("failed to parse default-remove-peer from PersistOptions's ttl storage", zap.Error(err)) + } + canSetRemovePeer := ok2 && err == nil + if canSetRemovePeer { + returnSC.RemovePeer = v + } + + if canSetAddPeer || canSetRemovePeer { + return returnSC + } + cfg.StoreLimit[storeID] = sc + o.SetScheduleConfig(cfg) + return o.GetScheduleConfig().StoreLimit[storeID] +} + // GetStoreLimitByType returns the limit of a store with a given type. func (o *PersistConfig) GetStoreLimitByType(storeID uint64, typ storelimit.Type) (returned float64) { + defer func() { + if typ == storelimit.RemovePeer { + returned = o.getTTLFloatOr(fmt.Sprintf("remove-peer-%v", storeID), returned) + } else if typ == storelimit.AddPeer { + returned = o.getTTLFloatOr(fmt.Sprintf("add-peer-%v", storeID), returned) + } + }() limit := o.GetStoreLimit(storeID) switch typ { case storelimit.AddPeer: @@ -550,20 +573,48 @@ func (o *PersistConfig) GetStoreLimitByType(storeID uint64, typ storelimit.Type) } } -// GetStoreLimit returns the limit of a store. -func (o *PersistConfig) GetStoreLimit(storeID uint64) (returnSC sc.StoreLimitConfig) { - if limit, ok := o.GetScheduleConfig().StoreLimit[storeID]; ok { - return limit +// GetMaxSnapshotCount returns the number of the max snapshot which is allowed to send. +func (o *PersistConfig) GetMaxSnapshotCount() uint64 { + return o.getTTLUintOr(sc.MaxSnapshotCountKey, o.GetScheduleConfig().MaxSnapshotCount) +} + +// GetMaxPendingPeerCount returns the number of the max pending peers. +func (o *PersistConfig) GetMaxPendingPeerCount() uint64 { + return o.getTTLUintOr(sc.MaxPendingPeerCountKey, o.GetScheduleConfig().MaxPendingPeerCount) +} + +// GetMaxMergeRegionSize returns the max region size. +func (o *PersistConfig) GetMaxMergeRegionSize() uint64 { + return o.getTTLUintOr(sc.MaxMergeRegionSizeKey, o.GetScheduleConfig().MaxMergeRegionSize) +} + +// GetMaxMergeRegionKeys returns the max number of keys. +// It returns size * 10000 if the key of max-merge-region-Keys doesn't exist. +func (o *PersistConfig) GetMaxMergeRegionKeys() uint64 { + keys, exist, err := o.getTTLUint(sc.MaxMergeRegionKeysKey) + if exist && err == nil { + return keys } - cfg := o.GetScheduleConfig().Clone() - sc := sc.StoreLimitConfig{ - AddPeer: sc.DefaultStoreLimit.GetDefaultStoreLimit(storelimit.AddPeer), - RemovePeer: sc.DefaultStoreLimit.GetDefaultStoreLimit(storelimit.RemovePeer), + size, exist, err := o.getTTLUint(sc.MaxMergeRegionSizeKey) + if exist && err == nil { + return size * 10000 } + return o.GetScheduleConfig().GetMaxMergeRegionKeys() +} - cfg.StoreLimit[storeID] = sc - o.SetScheduleConfig(cfg) - return o.GetScheduleConfig().StoreLimit[storeID] +// GetSchedulerMaxWaitingOperator returns the number of the max waiting operators. +func (o *PersistConfig) GetSchedulerMaxWaitingOperator() uint64 { + return o.getTTLUintOr(sc.SchedulerMaxWaitingOperatorKey, o.GetScheduleConfig().SchedulerMaxWaitingOperator) +} + +// IsLocationReplacementEnabled returns if location replace is enabled. +func (o *PersistConfig) IsLocationReplacementEnabled() bool { + return o.getTTLBoolOr(sc.EnableLocationReplacement, o.GetScheduleConfig().EnableLocationReplacement) +} + +// IsTikvRegionSplitEnabled returns whether tikv split region is disabled. +func (o *PersistConfig) IsTikvRegionSplitEnabled() bool { + return o.getTTLBoolOr(sc.EnableTiKVSplitRegion, o.GetScheduleConfig().EnableTiKVSplitRegion) } // SetAllStoresLimit sets all store limit for a given type and rate. @@ -680,11 +731,6 @@ func (o *PersistConfig) IsRaftKV2() bool { return o.GetStoreConfig().IsRaftKV2() } -// IsTikvRegionSplitEnabled returns whether tikv split region is disabled. -func (o *PersistConfig) IsTikvRegionSplitEnabled() bool { - return o.GetScheduleConfig().EnableTiKVSplitRegion -} - // TODO: implement the following methods // AddSchedulerCfg adds the scheduler configurations. @@ -710,3 +756,72 @@ func (o *PersistConfig) IsTraceRegionFlow() bool { func (o *PersistConfig) Persist(storage endpoint.ConfigStorage) error { return nil } + +func (o *PersistConfig) getTTLUint(key string) (uint64, bool, error) { + stringForm, ok := o.GetTTLData(key) + if !ok { + return 0, false, nil + } + r, err := strconv.ParseUint(stringForm, 10, 64) + return r, true, err +} + +func (o *PersistConfig) getTTLUintOr(key string, defaultValue uint64) uint64 { + if v, ok, err := o.getTTLUint(key); ok { + if err == nil { + return v + } + log.Warn("failed to parse "+key+" from PersistOptions's ttl storage", zap.Error(err)) + } + return defaultValue +} + +func (o *PersistConfig) getTTLBool(key string) (result bool, contains bool, err error) { + stringForm, ok := o.GetTTLData(key) + if !ok { + return + } + result, err = strconv.ParseBool(stringForm) + contains = true + return +} + +func (o *PersistConfig) getTTLBoolOr(key string, defaultValue bool) bool { + if v, ok, err := o.getTTLBool(key); ok { + if err == nil { + return v + } + log.Warn("failed to parse "+key+" from PersistOptions's ttl storage", zap.Error(err)) + } + return defaultValue +} + +func (o *PersistConfig) getTTLFloat(key string) (float64, bool, error) { + stringForm, ok := o.GetTTLData(key) + if !ok { + return 0, false, nil + } + r, err := strconv.ParseFloat(stringForm, 64) + return r, true, err +} + +func (o *PersistConfig) getTTLFloatOr(key string, defaultValue float64) float64 { + if v, ok, err := o.getTTLFloat(key); ok { + if err == nil { + return v + } + log.Warn("failed to parse "+key+" from PersistOptions's ttl storage", zap.Error(err)) + } + return defaultValue +} + +// GetTTLData returns if there is a TTL data for a given key. +func (o *PersistConfig) GetTTLData(key string) (string, bool) { + if o.ttl == nil { + return "", false + } + if result, ok := o.ttl.Get(key); ok { + return result.(string), ok + } + return "", false +} diff --git a/pkg/mcs/scheduling/server/config/watcher.go b/pkg/mcs/scheduling/server/config/watcher.go index 433933674ea..4ded93ceb1b 100644 --- a/pkg/mcs/scheduling/server/config/watcher.go +++ b/pkg/mcs/scheduling/server/config/watcher.go @@ -20,6 +20,7 @@ import ( "strings" "sync" "sync/atomic" + "time" "github.com/coreos/go-semver/semver" "github.com/pingcap/log" @@ -48,8 +49,11 @@ type Watcher struct { // - Value: configuration JSON. schedulerConfigPathPrefix string + ttlConfigPrefix string + etcdClient *clientv3.Client configWatcher *etcdutil.LoopWatcher + ttlConfigWatcher *etcdutil.LoopWatcher schedulerConfigWatcher *etcdutil.LoopWatcher // Some data, like the global schedule config, should be loaded into `PersistConfig`. @@ -82,6 +86,7 @@ func NewWatcher( ctx: ctx, cancel: cancel, configPath: endpoint.ConfigPath(clusterID), + ttlConfigPrefix: sc.TTLConfigPrefix, schedulerConfigPathPrefix: endpoint.SchedulerConfigPathPrefix(clusterID), etcdClient: etcdClient, PersistConfig: persistConfig, @@ -91,6 +96,10 @@ func NewWatcher( if err != nil { return nil, err } + err = cw.initializeTTLConfigWatcher() + if err != nil { + return nil, err + } err = cw.initializeSchedulerConfigWatcher() if err != nil { return nil, err @@ -143,6 +152,37 @@ func (cw *Watcher) initializeConfigWatcher() error { return cw.configWatcher.WaitLoad() } +func (cw *Watcher) initializeTTLConfigWatcher() error { + putFn := func(kv *mvccpb.KeyValue) error { + key := string(kv.Key)[len(sc.TTLConfigPrefix)+1:] + value := string(kv.Value) + leaseID := kv.Lease + resp, err := cw.etcdClient.TimeToLive(cw.ctx, clientv3.LeaseID(leaseID)) + if err != nil { + return err + } + log.Info("update scheduling ttl config", zap.String("key", key), zap.String("value", value)) + cw.ttl.PutWithTTL(key, value, time.Duration(resp.TTL)*time.Second) + return nil + } + deleteFn := func(kv *mvccpb.KeyValue) error { + key := string(kv.Key)[len(sc.TTLConfigPrefix)+1:] + cw.ttl.PutWithTTL(key, nil, 0) + return nil + } + postEventFn := func() error { + return nil + } + cw.ttlConfigWatcher = etcdutil.NewLoopWatcher( + cw.ctx, &cw.wg, + cw.etcdClient, + "scheduling-ttl-config-watcher", cw.ttlConfigPrefix, + putFn, deleteFn, postEventFn, clientv3.WithPrefix(), + ) + cw.ttlConfigWatcher.StartWatchLoop() + return cw.ttlConfigWatcher.WaitLoad() +} + func (cw *Watcher) initializeSchedulerConfigWatcher() error { prefixToTrim := cw.schedulerConfigPathPrefix + "/" putFn := func(kv *mvccpb.KeyValue) error { diff --git a/pkg/mcs/scheduling/server/server.go b/pkg/mcs/scheduling/server/server.go index c5b73dea5fc..a43cbbebd86 100644 --- a/pkg/mcs/scheduling/server/server.go +++ b/pkg/mcs/scheduling/server/server.go @@ -36,6 +36,7 @@ import ( "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" "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" @@ -46,6 +47,7 @@ import ( "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/member" "github.com/tikv/pd/pkg/schedule" + sc "github.com/tikv/pd/pkg/schedule/config" "github.com/tikv/pd/pkg/schedule/hbstream" "github.com/tikv/pd/pkg/schedule/schedulers" "github.com/tikv/pd/pkg/storage/endpoint" @@ -534,7 +536,7 @@ func CreateServer(ctx context.Context, cfg *config.Config) *Server { BaseServer: server.NewBaseServer(ctx), DiagnosticsServer: sysutil.NewDiagnosticsServer(cfg.Log.File.Filename), cfg: cfg, - persistConfig: config.NewPersistConfig(cfg), + persistConfig: config.NewPersistConfig(cfg, cache.NewStringTTL(ctx, sc.DefaultGCInterval, sc.DefaultTTL)), checkMembershipCh: make(chan struct{}, 1), } return svr diff --git a/pkg/schedule/checker/checker_controller.go b/pkg/schedule/checker/checker_controller.go index 68b794f417a..355226cd2d8 100644 --- a/pkg/schedule/checker/checker_controller.go +++ b/pkg/schedule/checker/checker_controller.go @@ -221,6 +221,11 @@ func (c *Controller) ClearSuspectKeyRanges() { c.suspectKeyRanges.Clear() } +// ClearSuspectRegions clears the suspect regions, only for unit test +func (c *Controller) ClearSuspectRegions() { + c.suspectRegions.Clear() +} + // IsPendingRegion returns true if the given region is in the pending list. func (c *Controller) IsPendingRegion(regionID uint64) bool { _, exist := c.ruleChecker.pendingList.Get(regionID) diff --git a/pkg/schedule/config/config.go b/pkg/schedule/config/config.go index c8fa62b8aff..27b8917d1bf 100644 --- a/pkg/schedule/config/config.go +++ b/pkg/schedule/config/config.go @@ -79,6 +79,34 @@ var ( DefaultTiFlashStoreLimit = StoreLimit{AddPeer: 30, RemovePeer: 30} ) +// The following consts are used to identify the config item that needs to set TTL. +const ( + // TTLConfigPrefix is the prefix of the config item that needs to set TTL. + TTLConfigPrefix = "/config/ttl" + + MaxSnapshotCountKey = "schedule.max-snapshot-count" + MaxMergeRegionSizeKey = "schedule.max-merge-region-size" + MaxPendingPeerCountKey = "schedule.max-pending-peer-count" + MaxMergeRegionKeysKey = "schedule.max-merge-region-keys" + LeaderScheduleLimitKey = "schedule.leader-schedule-limit" + RegionScheduleLimitKey = "schedule.region-schedule-limit" + WitnessScheduleLimitKey = "schedule.witness-schedule-limit" + ReplicaRescheduleLimitKey = "schedule.replica-schedule-limit" + MergeScheduleLimitKey = "schedule.merge-schedule-limit" + HotRegionScheduleLimitKey = "schedule.hot-region-schedule-limit" + SchedulerMaxWaitingOperatorKey = "schedule.scheduler-max-waiting-operator" + EnableLocationReplacement = "schedule.enable-location-replacement" + DefaultAddPeer = "default-add-peer" + DefaultRemovePeer = "default-remove-peer" + + // EnableTiKVSplitRegion is the option to enable tikv split region. + // it's related to schedule, but it's not an explicit config + EnableTiKVSplitRegion = "schedule.enable-tikv-split-region" + + DefaultGCInterval = 5 * time.Second + DefaultTTL = 5 * time.Minute +) + // StoreLimit is the default limit of adding peer and removing peer when putting stores. type StoreLimit struct { mu syncutil.RWMutex diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index c20abfc5f79..0c1b017d7aa 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -1224,14 +1224,12 @@ func waitForPrimariesServing( re *require.Assertions, mgrs []*KeyspaceGroupManager, ids []uint32, ) { testutil.Eventually(re, func() bool { - for i := 0; i < 100; i++ { - for j, id := range ids { - if member, err := mgrs[j].GetElectionMember(id, id); err != nil || !member.IsLeader() { - return false - } - if _, _, err := mgrs[j].HandleTSORequest(mgrs[j].ctx, id, id, GlobalDCLocation, 1); err != nil { - return false - } + for j, id := range ids { + if member, err := mgrs[j].GetElectionMember(id, id); err != nil || member == nil || !member.IsLeader() { + return false + } + if _, _, err := mgrs[j].HandleTSORequest(mgrs[j].ctx, id, id, GlobalDCLocation, 1); err != nil { + return false } } return true diff --git a/pkg/utils/grpcutil/grpcutil.go b/pkg/utils/grpcutil/grpcutil.go index 44d45ff4c70..a001ec4bd03 100644 --- a/pkg/utils/grpcutil/grpcutil.go +++ b/pkg/utils/grpcutil/grpcutil.go @@ -163,6 +163,7 @@ func GetForwardedHost(ctx context.Context) string { md, ok := metadata.FromIncomingContext(ctx) if !ok { log.Debug("failed to get forwarding metadata") + return "" } if t, ok := md[ForwardMetadataKey]; ok { return t[0] diff --git a/pkg/utils/tsoutil/tsoutil.go b/pkg/utils/tsoutil/tsoutil.go index 796012ae031..43d8b09aa49 100644 --- a/pkg/utils/tsoutil/tsoutil.go +++ b/pkg/utils/tsoutil/tsoutil.go @@ -25,6 +25,11 @@ const ( logicalBits = (1 << physicalShiftBits) - 1 ) +// TimeToTS converts a `time.Time` to an `uint64` TS. +func TimeToTS(t time.Time) uint64 { + return ComposeTS(t.UnixNano()/int64(time.Millisecond), 0) +} + // ParseTS parses the ts to (physical,logical). func ParseTS(ts uint64) (time.Time, uint64) { physical, logical := ParseTSUint64(ts) diff --git a/scripts/ci-subtask.sh b/scripts/ci-subtask.sh index a2e396088d6..dd29895327b 100755 --- a/scripts/ci-subtask.sh +++ b/scripts/ci-subtask.sh @@ -29,11 +29,22 @@ else weight() { [[ $1 == "github.com/tikv/pd/server/api" ]] && return 30 [[ $1 == "github.com/tikv/pd/pkg/schedule" ]] && return 30 - [[ $1 == "pd/tests/server/api" ]] && return 30 + [[ $1 == "github.com/tikv/pd/pkg/core" ]] && return 30 + [[ $1 == "github.com/tikv/pd/tests/server/api" ]] && return 30 [[ $1 =~ "pd/tests" ]] && return 5 return 1 } + # Create an associative array to store the weight of each task. + declare -A task_weights + for t in ${tasks[@]}; do + weight $t + task_weights[$t]=$? + done + + # Sort tasks by weight in descending order. + tasks=($(printf "%s\n" "${tasks[@]}" | sort -rn)) + scores=($(seq "$1" | xargs -I{} echo 0)) res=() @@ -42,8 +53,7 @@ else for i in ${!scores[@]}; do [[ ${scores[i]} -lt ${scores[$min_i]} ]] && min_i=$i done - weight $t - scores[$min_i]=$((${scores[$min_i]} + $?)) + scores[$min_i]=$((${scores[$min_i]} + ${task_weights[$t]})) [[ $(($min_i + 1)) -eq $2 ]] && res+=($t) done printf "%s " "${res[@]}" diff --git a/server/apiv2/handlers/keyspace.go b/server/apiv2/handlers/keyspace.go index 9602cc863ef..c2802bb939d 100644 --- a/server/apiv2/handlers/keyspace.go +++ b/server/apiv2/handlers/keyspace.go @@ -113,7 +113,7 @@ func LoadKeyspace(c *gin.Context) { if value, ok := c.GetQuery("force_refresh_group_id"); ok && value == "true" { groupManager := svr.GetKeyspaceGroupManager() if groupManager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, managerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } // keyspace has been checked in LoadKeyspace, so no need to check again. diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index a580b21f705..a9f042687f6 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -30,7 +30,8 @@ import ( "github.com/tikv/pd/server/apiv2/middlewares" ) -const groupManagerUninitializedErr = "keyspace group manager is not initialized" +// GroupManagerUninitializedErr is the error message for uninitialized keyspace group manager. +const GroupManagerUninitializedErr = "keyspace group manager is not initialized" // RegisterTSOKeyspaceGroup registers keyspace group handlers to the server. func RegisterTSOKeyspaceGroup(r *gin.RouterGroup) { @@ -78,7 +79,7 @@ func CreateKeyspaceGroups(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } err = manager.CreateKeyspaceGroups(createParams.KeyspaceGroups) @@ -101,7 +102,7 @@ func GetKeyspaceGroups(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } keyspaceGroups, err := manager.GetKeyspaceGroups(scanStart, scanLimit) @@ -152,7 +153,7 @@ func GetKeyspaceGroupByID(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } @@ -189,7 +190,7 @@ func DeleteKeyspaceGroupByID(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } kg, err := manager.DeleteKeyspaceGroupByID(id) @@ -250,7 +251,7 @@ func SplitKeyspaceGroupByID(c *gin.Context) { } groupManager := svr.GetKeyspaceGroupManager() if groupManager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } @@ -289,7 +290,7 @@ func FinishSplitKeyspaceByID(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } err = manager.FinishSplitKeyspaceByID(id) @@ -337,7 +338,7 @@ func MergeKeyspaceGroups(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) groupManager := svr.GetKeyspaceGroupManager() if groupManager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } // Merge keyspace group. @@ -364,7 +365,7 @@ func FinishMergeKeyspaceByID(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } err = manager.FinishMergeKeyspaceByID(id) @@ -390,7 +391,7 @@ func AllocNodesForKeyspaceGroup(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } allocParams := &AllocNodesForKeyspaceGroupParams{} @@ -437,7 +438,7 @@ func SetNodesForKeyspaceGroup(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } setParams := &SetNodesForKeyspaceGroupParams{} @@ -493,7 +494,7 @@ func SetPriorityForKeyspaceGroup(c *gin.Context) { svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { - c.AbortWithStatusJSON(http.StatusInternalServerError, groupManagerUninitializedErr) + c.AbortWithStatusJSON(http.StatusInternalServerError, GroupManagerUninitializedErr) return } setParams := &SetPriorityForKeyspaceGroupParams{} diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index 1c3d8a03a98..ec8ca3a0d65 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -2241,7 +2241,9 @@ func (c *RaftCluster) SetMinResolvedTS(storeID, minResolvedTS uint64) error { return nil } -func (c *RaftCluster) checkAndUpdateMinResolvedTS() (uint64, bool) { +// CheckAndUpdateMinResolvedTS checks and updates the min resolved ts of the cluster. +// This is exported for testing purpose. +func (c *RaftCluster) CheckAndUpdateMinResolvedTS() (uint64, bool) { c.Lock() defer c.Unlock() @@ -2284,7 +2286,7 @@ func (c *RaftCluster) runMinResolvedTSJob() { case <-ticker.C: interval = c.opt.GetMinResolvedTSPersistenceInterval() if interval != 0 { - if current, needPersist := c.checkAndUpdateMinResolvedTS(); needPersist { + if current, needPersist := c.CheckAndUpdateMinResolvedTS(); needPersist { c.storage.SaveMinResolvedTS(current) } } else { diff --git a/server/cluster/scheduling_controller.go b/server/cluster/scheduling_controller.go index 5e8cb8462df..a36e7159cfd 100644 --- a/server/cluster/scheduling_controller.go +++ b/server/cluster/scheduling_controller.go @@ -438,6 +438,13 @@ func (sc *schedulingController) ClearSuspectKeyRanges() { sc.coordinator.GetCheckerController().ClearSuspectKeyRanges() } +// ClearSuspectRegions clears the suspect regions, only for unit test +func (sc *schedulingController) ClearSuspectRegions() { + sc.mu.RLock() + defer sc.mu.RUnlock() + sc.coordinator.GetCheckerController().ClearSuspectRegions() +} + // AddSuspectKeyRange adds the key range with the its ruleID as the key // The instance of each keyRange is like following format: // [2][]byte: start key/end key diff --git a/server/config/persist_options.go b/server/config/persist_options.go index 49a44449a22..ae9047c626b 100644 --- a/server/config/persist_options.go +++ b/server/config/persist_options.go @@ -216,38 +216,21 @@ func (o *PersistOptions) SetMaxReplicas(replicas int) { o.SetReplicationConfig(v) } -const ( - maxSnapshotCountKey = "schedule.max-snapshot-count" - maxMergeRegionSizeKey = "schedule.max-merge-region-size" - maxPendingPeerCountKey = "schedule.max-pending-peer-count" - maxMergeRegionKeysKey = "schedule.max-merge-region-keys" - leaderScheduleLimitKey = "schedule.leader-schedule-limit" - regionScheduleLimitKey = "schedule.region-schedule-limit" - witnessScheduleLimitKey = "schedule.witness-schedule-limit" - replicaRescheduleLimitKey = "schedule.replica-schedule-limit" - mergeScheduleLimitKey = "schedule.merge-schedule-limit" - hotRegionScheduleLimitKey = "schedule.hot-region-schedule-limit" - schedulerMaxWaitingOperatorKey = "schedule.scheduler-max-waiting-operator" - enableLocationReplacement = "schedule.enable-location-replacement" - // it's related to schedule, but it's not an explicit config - enableTiKVSplitRegion = "schedule.enable-tikv-split-region" -) - var supportedTTLConfigs = []string{ - maxSnapshotCountKey, - maxMergeRegionSizeKey, - maxPendingPeerCountKey, - maxMergeRegionKeysKey, - leaderScheduleLimitKey, - regionScheduleLimitKey, - replicaRescheduleLimitKey, - mergeScheduleLimitKey, - hotRegionScheduleLimitKey, - schedulerMaxWaitingOperatorKey, - enableLocationReplacement, - enableTiKVSplitRegion, - "default-add-peer", - "default-remove-peer", + sc.MaxSnapshotCountKey, + sc.MaxMergeRegionSizeKey, + sc.MaxPendingPeerCountKey, + sc.MaxMergeRegionKeysKey, + sc.LeaderScheduleLimitKey, + sc.RegionScheduleLimitKey, + sc.ReplicaRescheduleLimitKey, + sc.MergeScheduleLimitKey, + sc.HotRegionScheduleLimitKey, + sc.SchedulerMaxWaitingOperatorKey, + sc.EnableLocationReplacement, + sc.EnableTiKVSplitRegion, + sc.DefaultAddPeer, + sc.DefaultRemovePeer, } // IsSupportedTTLConfig checks whether a key is a supported config item with ttl @@ -262,27 +245,27 @@ func IsSupportedTTLConfig(key string) bool { // GetMaxSnapshotCount returns the number of the max snapshot which is allowed to send. func (o *PersistOptions) GetMaxSnapshotCount() uint64 { - return o.getTTLUintOr(maxSnapshotCountKey, o.GetScheduleConfig().MaxSnapshotCount) + return o.getTTLUintOr(sc.MaxSnapshotCountKey, o.GetScheduleConfig().MaxSnapshotCount) } // GetMaxPendingPeerCount returns the number of the max pending peers. func (o *PersistOptions) GetMaxPendingPeerCount() uint64 { - return o.getTTLUintOr(maxPendingPeerCountKey, o.GetScheduleConfig().MaxPendingPeerCount) + return o.getTTLUintOr(sc.MaxPendingPeerCountKey, o.GetScheduleConfig().MaxPendingPeerCount) } // GetMaxMergeRegionSize returns the max region size. func (o *PersistOptions) GetMaxMergeRegionSize() uint64 { - return o.getTTLUintOr(maxMergeRegionSizeKey, o.GetScheduleConfig().MaxMergeRegionSize) + return o.getTTLUintOr(sc.MaxMergeRegionSizeKey, o.GetScheduleConfig().MaxMergeRegionSize) } // GetMaxMergeRegionKeys returns the max number of keys. // It returns size * 10000 if the key of max-merge-region-Keys doesn't exist. func (o *PersistOptions) GetMaxMergeRegionKeys() uint64 { - keys, exist, err := o.getTTLUint(maxMergeRegionKeysKey) + keys, exist, err := o.getTTLUint(sc.MaxMergeRegionKeysKey) if exist && err == nil { return keys } - size, exist, err := o.getTTLUint(maxMergeRegionSizeKey) + size, exist, err := o.getTTLUint(sc.MaxMergeRegionSizeKey) if exist && err == nil { return size * 10000 } @@ -424,32 +407,32 @@ func (o *PersistOptions) GetMaxStorePreparingTime() time.Duration { // GetLeaderScheduleLimit returns the limit for leader schedule. func (o *PersistOptions) GetLeaderScheduleLimit() uint64 { - return o.getTTLUintOr(leaderScheduleLimitKey, o.GetScheduleConfig().LeaderScheduleLimit) + return o.getTTLUintOr(sc.LeaderScheduleLimitKey, o.GetScheduleConfig().LeaderScheduleLimit) } // GetRegionScheduleLimit returns the limit for region schedule. func (o *PersistOptions) GetRegionScheduleLimit() uint64 { - return o.getTTLUintOr(regionScheduleLimitKey, o.GetScheduleConfig().RegionScheduleLimit) + return o.getTTLUintOr(sc.RegionScheduleLimitKey, o.GetScheduleConfig().RegionScheduleLimit) } // GetWitnessScheduleLimit returns the limit for region schedule. func (o *PersistOptions) GetWitnessScheduleLimit() uint64 { - return o.getTTLUintOr(witnessScheduleLimitKey, o.GetScheduleConfig().WitnessScheduleLimit) + return o.getTTLUintOr(sc.WitnessScheduleLimitKey, o.GetScheduleConfig().WitnessScheduleLimit) } // GetReplicaScheduleLimit returns the limit for replica schedule. func (o *PersistOptions) GetReplicaScheduleLimit() uint64 { - return o.getTTLUintOr(replicaRescheduleLimitKey, o.GetScheduleConfig().ReplicaScheduleLimit) + return o.getTTLUintOr(sc.ReplicaRescheduleLimitKey, o.GetScheduleConfig().ReplicaScheduleLimit) } // GetMergeScheduleLimit returns the limit for merge schedule. func (o *PersistOptions) GetMergeScheduleLimit() uint64 { - return o.getTTLUintOr(mergeScheduleLimitKey, o.GetScheduleConfig().MergeScheduleLimit) + return o.getTTLUintOr(sc.MergeScheduleLimitKey, o.GetScheduleConfig().MergeScheduleLimit) } // GetHotRegionScheduleLimit returns the limit for hot region schedule. func (o *PersistOptions) GetHotRegionScheduleLimit() uint64 { - return o.getTTLUintOr(hotRegionScheduleLimitKey, o.GetScheduleConfig().HotRegionScheduleLimit) + return o.getTTLUintOr(sc.HotRegionScheduleLimitKey, o.GetScheduleConfig().HotRegionScheduleLimit) } // GetStoreLimit returns the limit of a store. @@ -552,7 +535,7 @@ func (o *PersistOptions) GetRegionScoreFormulaVersion() string { // GetSchedulerMaxWaitingOperator returns the number of the max waiting operators. func (o *PersistOptions) GetSchedulerMaxWaitingOperator() uint64 { - return o.getTTLUintOr(schedulerMaxWaitingOperatorKey, o.GetScheduleConfig().SchedulerMaxWaitingOperator) + return o.getTTLUintOr(sc.SchedulerMaxWaitingOperatorKey, o.GetScheduleConfig().SchedulerMaxWaitingOperator) } // GetLeaderSchedulePolicy is to get leader schedule policy. @@ -622,12 +605,12 @@ func (o *PersistOptions) IsRemoveExtraReplicaEnabled() bool { // IsLocationReplacementEnabled returns if location replace is enabled. func (o *PersistOptions) IsLocationReplacementEnabled() bool { - return o.getTTLBoolOr(enableLocationReplacement, o.GetScheduleConfig().EnableLocationReplacement) + return o.getTTLBoolOr(sc.EnableLocationReplacement, o.GetScheduleConfig().EnableLocationReplacement) } // IsTikvRegionSplitEnabled returns whether tikv split region is disabled. func (o *PersistOptions) IsTikvRegionSplitEnabled() bool { - return o.getTTLBoolOr(enableTiKVSplitRegion, o.GetScheduleConfig().EnableTiKVSplitRegion) + return o.getTTLBoolOr(sc.EnableTiKVSplitRegion, o.GetScheduleConfig().EnableTiKVSplitRegion) } // GetMaxMovableHotPeerSize returns the max movable hot peer size. @@ -852,16 +835,22 @@ func (o *PersistOptions) GetMinResolvedTSPersistenceInterval() time.Duration { return o.GetPDServerConfig().MinResolvedTSPersistenceInterval.Duration } -const ttlConfigPrefix = "/config/ttl" - // SetTTLData set temporary configuration func (o *PersistOptions) SetTTLData(parCtx context.Context, client *clientv3.Client, key string, value string, ttl time.Duration) error { if o.ttl == nil { - o.ttl = cache.NewStringTTL(parCtx, time.Second*5, time.Minute*5) + o.ttl = cache.NewStringTTL(parCtx, sc.DefaultGCInterval, sc.DefaultTTL) } - _, err := etcdutil.EtcdKVPutWithTTL(parCtx, client, ttlConfigPrefix+"/"+key, value, int64(ttl.Seconds())) - if err != nil { - return err + if ttl != 0 { + // the minimum ttl is 5 seconds, if the given ttl is less than 5 seconds, we will use 5 seconds instead. + _, err := etcdutil.EtcdKVPutWithTTL(parCtx, client, sc.TTLConfigPrefix+"/"+key, value, int64(ttl.Seconds())) + if err != nil { + return err + } + } else { + _, err := client.Delete(parCtx, sc.TTLConfigPrefix+"/"+key) + if err != nil { + return err + } } o.ttl.PutWithTTL(key, value, ttl) return nil @@ -938,15 +927,15 @@ func (o *PersistOptions) GetTTLData(key string) (string, bool) { // LoadTTLFromEtcd loads temporary configuration which was persisted into etcd func (o *PersistOptions) LoadTTLFromEtcd(ctx context.Context, client *clientv3.Client) error { - resps, err := etcdutil.EtcdKVGet(client, ttlConfigPrefix, clientv3.WithPrefix()) + resps, err := etcdutil.EtcdKVGet(client, sc.TTLConfigPrefix, clientv3.WithPrefix()) if err != nil { return err } if o.ttl == nil { - o.ttl = cache.NewStringTTL(ctx, time.Second*5, time.Minute*5) + o.ttl = cache.NewStringTTL(ctx, sc.DefaultGCInterval, sc.DefaultTTL) } for _, resp := range resps.Kvs { - key := string(resp.Key)[len(ttlConfigPrefix)+1:] + key := string(resp.Key)[len(sc.TTLConfigPrefix)+1:] value := string(resp.Value) leaseID := resp.Lease resp, err := client.TimeToLive(ctx, clientv3.LeaseID(leaseID)) @@ -1035,6 +1024,7 @@ func (o *PersistOptions) IsRaftKV2() bool { } // SetRegionBucketEnabled sets if the region bucket is enabled. +// only used for test. func (o *PersistOptions) SetRegionBucketEnabled(enabled bool) { cfg := o.GetStoreConfig().Clone() cfg.SetRegionBucketEnabled(enabled) diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index a4aca195f3f..799901ff2e3 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -15,7 +15,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20231018065736-c0689aded40c github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.3 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 @@ -50,9 +50,11 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect + github.com/bytedance/sonic v1.9.1 // indirect github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cenkalti/backoff/v4 v4.0.2 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect @@ -60,23 +62,24 @@ require ( github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 // indirect github.com/elliotchance/pie/v2 v2.1.0 // indirect github.com/fogleman/gg v1.3.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/gin-contrib/cors v1.4.0 // indirect github.com/gin-contrib/gzip v0.0.1 // indirect github.com/gin-contrib/pprof v1.4.0 // indirect github.com/gin-contrib/sse v0.1.0 // indirect - github.com/gin-gonic/gin v1.8.1 // indirect + github.com/gin-gonic/gin v1.9.1 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect github.com/go-openapi/jsonreference v0.19.6 // indirect github.com/go-openapi/spec v0.20.4 // indirect github.com/go-openapi/swag v0.19.15 // indirect - github.com/go-playground/locales v0.14.0 // indirect - github.com/go-playground/universal-translator v0.18.0 // indirect - github.com/go-playground/validator/v10 v10.10.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.0.9 // indirect - github.com/goccy/go-json v0.9.7 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect @@ -100,11 +103,12 @@ 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/cpuid/v2 v2.2.4 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/leodido/go-urn v1.2.1 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect github.com/mailru/easyjson v0.7.6 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/mattn/go-sqlite3 v1.14.15 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/minio/sio v0.3.0 // indirect @@ -112,14 +116,14 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pelletier/go-toml/v2 v2.0.1 // indirect + github.com/pelletier/go-toml/v2 v2.0.8 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 // indirect github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -147,7 +151,8 @@ require ( github.com/tklauser/go-sysconf v0.3.11 // indirect github.com/tklauser/numcpus v0.6.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 // indirect - github.com/ugorji/go/codec v1.2.7 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/ugorji/go/codec v1.2.11 // indirect github.com/unrolled/render v1.0.1 // indirect github.com/urfave/negroni v0.3.0 // indirect github.com/vmihailenco/msgpack/v5 v5.3.5 // indirect @@ -159,6 +164,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect + golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.14.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.5.0 // indirect diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index ef9c4d2a5f3..e13da5d8375 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -68,6 +68,9 @@ github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4Yn github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpfpgJhvkuYWBmOYbo= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.0.2 h1:JIufpQLbh4DkbQoii76ItQIUFzevQSqOLZca4eamEDs= @@ -77,6 +80,9 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= 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= @@ -123,6 +129,8 @@ github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5Kwzbycv github.com/fogleman/gg v1.3.0 h1:/7zJX8F6AaYQc57WQCyN9cAIz+4bCJGO9B+dyW29am8= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= +github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/cors v1.4.0 h1:oJ6gwtUl3lqV0WEIwM/LxPF1QZ5qe2lGWdY2+bz7y0g= github.com/gin-contrib/cors v1.4.0/go.mod h1:bs9pNM0x/UsmHPBWT2xZz9ROh8xYjYkiURUfmBoMlcs= @@ -135,8 +143,9 @@ github.com/gin-contrib/sse v0.1.0 h1:Y/yl/+YNO8GZSjAhjMsSuLt29uWRFHdHYUb5lYOV9qE github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= -github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= +github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= +github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -155,17 +164,21 @@ github.com/go-openapi/spec v0.20.4/go.mod h1:faYFR1CvsJZ0mNsmsphTMSoRrNV3TEDoAM7 github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.15 h1:D2NRCBzS9/pEY3gP9Nl8aDqGUcPFrwG2p+CNFrLyrCM= github.com/go-openapi/swag v0.19.15/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= -github.com/go-playground/assert/v2 v2.0.1 h1:MsBgLAaY856+nPRTKrp3/OZK38U/wa0CcBYNjji3q3A= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/locales v0.14.0 h1:u50s323jtVGugKlcYeyzC0etD1HifMjqmJqb8WugfUU= github.com/go-playground/locales v0.14.0/go.mod h1:sawfccIbzZTqEDETgFXqTho0QybSa7l++s0DH+LDiLs= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= -github.com/go-playground/universal-translator v0.18.0 h1:82dyy6p4OuJq4/CByFNOn/jYrnRPArHwAcmLoJZxyho= github.com/go-playground/universal-translator v0.18.0/go.mod h1:UvRDBj+xPUEGrFYl+lu/H90nyDXpg0fqeB/AQUGNTVA= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= -github.com/go-playground/validator/v10 v10.10.0 h1:I7mrTYv78z8k8VXa/qJlOlEXn/nBh+BF8dHX5nt/dr0= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= +github.com/go-playground/validator/v10 v10.14.0 h1:vgvQWe3XCz3gIeFDm/HnTIbj6UGmg/+t63MyGU2n5js= +github.com/go-playground/validator/v10 v10.14.0/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -173,8 +186,9 @@ github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9 github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-graphviz v0.0.9 h1:s/FMMJ1Joj6La3S5ApO3Jk2cwM4LpXECC2muFx3IPQQ= github.com/goccy/go-graphviz v0.0.9/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= -github.com/goccy/go-json v0.9.7 h1:IcB+Aqpx/iMHu5Yooh7jEzJk1JZ7Pjtmys2ukPr7EeM= github.com/goccy/go-json v0.9.7/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -312,6 +326,9 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= 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/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -325,8 +342,9 @@ 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/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= -github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= @@ -338,8 +356,9 @@ github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJ github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.14.15 h1:vfoHhTN1af61xCRSWzFIWzx2YskyMTwHLrExkBOjvxI= github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= @@ -383,8 +402,9 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml/v2 v2.0.1 h1:8e3L2cCQzLFi2CR4g7vGFuFxX7Jl1kKX8gW+iV0GUKU= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= +github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -410,8 +430,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 h1:wnHt7ETIB0vm+gbLx8QhcIEmRtrT4QlWlfpcI9vjxOk= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537/go.mod h1:EZ90+V5S4TttbYag6oKZ3jcNKRwZe1Mc9vXwOt9JBYw= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e h1:SJUSDejvKtj9vSh5ptRHh4iMrvPV3oKO8yp6/SYE8vc= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -508,8 +528,10 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ 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.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -535,13 +557,16 @@ github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go v1.2.7/go.mod h1:nF9osbDWLy6bDVv/Rtoh6QgnvNDpmCalQV5urGCCS6M= github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= +github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= +github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= @@ -599,6 +624,9 @@ go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -708,6 +736,7 @@ golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/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-20220704084225-05e143d24a9e/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.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -842,6 +871,7 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= moul.io/zapgorm2 v1.1.0 h1:qwAlMBYf+qJkJ7PAzJl4oCe6eS6QGiKAXUPeis0+RBE= moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/tests/integrations/client/http_client_test.go b/tests/integrations/client/http_client_test.go index 213aa57de46..a007b893187 100644 --- a/tests/integrations/client/http_client_test.go +++ b/tests/integrations/client/http_client_test.go @@ -17,13 +17,19 @@ package client_test import ( "context" "math" + "net/http" "sort" "testing" + "time" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" pd "github.com/tikv/pd/client/http" + "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/labeler" "github.com/tikv/pd/pkg/schedule/placement" + "github.com/tikv/pd/pkg/utils/testutil" + "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/tests" ) @@ -49,8 +55,16 @@ func (suite *httpClientTestSuite) SetupSuite() { re.NoError(err) leader := suite.cluster.WaitLeader() re.NotEmpty(leader) - err = suite.cluster.GetLeaderServer().BootstrapCluster() + leaderServer := suite.cluster.GetLeaderServer() + err = leaderServer.BootstrapCluster() re.NoError(err) + for _, region := range []*core.RegionInfo{ + core.NewTestRegionInfo(10, 1, []byte("a1"), []byte("a2")), + core.NewTestRegionInfo(11, 1, []byte("a2"), []byte("a3")), + } { + err := leaderServer.GetRaftCluster().HandleRegionHeartbeat(region) + re.NoError(err) + } var ( testServers = suite.cluster.GetServers() endpoints = make([]string, 0, len(testServers)) @@ -67,23 +81,79 @@ func (suite *httpClientTestSuite) TearDownSuite() { suite.cluster.Destroy() } +func (suite *httpClientTestSuite) TestMeta() { + re := suite.Require() + region, err := suite.client.GetRegionByID(suite.ctx, 10) + re.NoError(err) + re.Equal(int64(10), region.ID) + re.Equal(core.HexRegionKeyStr([]byte("a1")), region.StartKey) + re.Equal(core.HexRegionKeyStr([]byte("a2")), region.EndKey) + region, err = suite.client.GetRegionByKey(suite.ctx, []byte("a2")) + re.NoError(err) + re.Equal(int64(11), region.ID) + re.Equal(core.HexRegionKeyStr([]byte("a2")), region.StartKey) + re.Equal(core.HexRegionKeyStr([]byte("a3")), region.EndKey) + regions, err := suite.client.GetRegions(suite.ctx) + re.NoError(err) + re.Equal(int64(2), regions.Count) + re.Len(regions.Regions, 2) + regions, err = suite.client.GetRegionsByKeyRange(suite.ctx, pd.NewKeyRange([]byte("a1"), []byte("a3")), -1) + re.NoError(err) + re.Equal(int64(2), regions.Count) + re.Len(regions.Regions, 2) + regions, err = suite.client.GetRegionsByStoreID(suite.ctx, 1) + re.NoError(err) + re.Equal(int64(2), regions.Count) + re.Len(regions.Regions, 2) + regionStats, err := suite.client.GetRegionStatusByKeyRange(suite.ctx, pd.NewKeyRange([]byte("a1"), []byte("a3"))) + re.NoError(err) + re.Equal(2, regionStats.Count) + hotReadRegions, err := suite.client.GetHotReadRegions(suite.ctx) + re.NoError(err) + re.Len(hotReadRegions.AsPeer, 1) + re.Len(hotReadRegions.AsLeader, 1) + hotWriteRegions, err := suite.client.GetHotWriteRegions(suite.ctx) + re.NoError(err) + re.Len(hotWriteRegions.AsPeer, 1) + re.Len(hotWriteRegions.AsLeader, 1) + historyHorRegions, err := suite.client.GetHistoryHotRegions(suite.ctx, &pd.HistoryHotRegionsRequest{ + StartTime: 0, + EndTime: time.Now().AddDate(0, 0, 1).UnixNano() / int64(time.Millisecond), + }) + re.NoError(err) + re.Len(historyHorRegions.HistoryHotRegion, 0) + store, err := suite.client.GetStores(suite.ctx) + re.NoError(err) + re.Equal(1, store.Count) + re.Len(store.Stores, 1) +} + func (suite *httpClientTestSuite) TestGetMinResolvedTSByStoresIDs() { re := suite.Require() - // Get the cluster-level min resolved TS. + testMinResolvedTS := tsoutil.TimeToTS(time.Now()) + raftCluster := suite.cluster.GetLeaderServer().GetRaftCluster() + err := raftCluster.SetMinResolvedTS(1, testMinResolvedTS) + re.NoError(err) + // Make sure the min resolved TS is updated. + testutil.Eventually(re, func() bool { + minResolvedTS, _ := raftCluster.CheckAndUpdateMinResolvedTS() + return minResolvedTS == testMinResolvedTS + }) + // Wait for the cluster-level min resolved TS to be initialized. minResolvedTS, storeMinResolvedTSMap, err := suite.client.GetMinResolvedTSByStoresIDs(suite.ctx, nil) re.NoError(err) - re.Greater(minResolvedTS, uint64(0)) + re.Equal(testMinResolvedTS, minResolvedTS) re.Empty(storeMinResolvedTSMap) // Get the store-level min resolved TS. minResolvedTS, storeMinResolvedTSMap, err = suite.client.GetMinResolvedTSByStoresIDs(suite.ctx, []uint64{1}) re.NoError(err) - re.Greater(minResolvedTS, uint64(0)) + re.Equal(testMinResolvedTS, minResolvedTS) re.Len(storeMinResolvedTSMap, 1) re.Equal(minResolvedTS, storeMinResolvedTSMap[1]) // Get the store-level min resolved TS with an invalid store ID. minResolvedTS, storeMinResolvedTSMap, err = suite.client.GetMinResolvedTSByStoresIDs(suite.ctx, []uint64{1, 2}) re.NoError(err) - re.Greater(minResolvedTS, uint64(0)) + re.Equal(testMinResolvedTS, minResolvedTS) re.Len(storeMinResolvedTSMap, 2) re.Equal(minResolvedTS, storeMinResolvedTSMap[1]) re.Equal(uint64(math.MaxUint64), storeMinResolvedTSMap[2]) @@ -98,15 +168,15 @@ func (suite *httpClientTestSuite) TestRule() { bundle, err := suite.client.GetPlacementRuleBundleByGroup(suite.ctx, placement.DefaultGroupID) re.NoError(err) re.Equal(bundles[0], bundle) - rules, err := suite.client.GetPlacementRulesByGroup(suite.ctx, placement.DefaultGroupID) - re.NoError(err) - re.Len(rules, 1) - re.Equal(placement.DefaultGroupID, rules[0].GroupID) - re.Equal(placement.DefaultRuleID, rules[0].ID) - re.Equal(pd.Voter, rules[0].Role) - re.Equal(3, rules[0].Count) + // Check if we have the default rule. + suite.checkRule(re, &pd.Rule{ + GroupID: placement.DefaultGroupID, + ID: placement.DefaultRuleID, + Role: pd.Voter, + Count: 3, + }, 1, true) // Should be the same as the rules in the bundle. - re.Equal(bundle.Rules, rules) + suite.checkRule(re, bundle.Rules[0], 1, true) testRule := &pd.Rule{ GroupID: placement.DefaultGroupID, ID: "test", @@ -115,20 +185,24 @@ func (suite *httpClientTestSuite) TestRule() { } err = suite.client.SetPlacementRule(suite.ctx, testRule) re.NoError(err) - rules, err = suite.client.GetPlacementRulesByGroup(suite.ctx, placement.DefaultGroupID) - re.NoError(err) - re.Len(rules, 2) - re.Equal(placement.DefaultGroupID, rules[1].GroupID) - re.Equal("test", rules[1].ID) - re.Equal(pd.Voter, rules[1].Role) - re.Equal(3, rules[1].Count) + suite.checkRule(re, testRule, 2, true) err = suite.client.DeletePlacementRule(suite.ctx, placement.DefaultGroupID, "test") re.NoError(err) - rules, err = suite.client.GetPlacementRulesByGroup(suite.ctx, placement.DefaultGroupID) + suite.checkRule(re, testRule, 1, false) + testRuleOp := &pd.RuleOp{ + Rule: testRule, + Action: pd.RuleOpAdd, + } + err = suite.client.SetPlacementRuleInBatch(suite.ctx, []*pd.RuleOp{testRuleOp}) + re.NoError(err) + suite.checkRule(re, testRule, 2, true) + testRuleOp = &pd.RuleOp{ + Rule: testRule, + Action: pd.RuleOpDel, + } + err = suite.client.SetPlacementRuleInBatch(suite.ctx, []*pd.RuleOp{testRuleOp}) re.NoError(err) - re.Len(rules, 1) - re.Equal(placement.DefaultGroupID, rules[0].GroupID) - re.Equal(placement.DefaultRuleID, rules[0].ID) + suite.checkRule(re, testRule, 1, false) err = suite.client.SetPlacementRuleBundles(suite.ctx, []*pd.GroupBundle{ { ID: placement.DefaultGroupID, @@ -136,14 +210,63 @@ func (suite *httpClientTestSuite) TestRule() { }, }, true) re.NoError(err) - bundles, err = suite.client.GetAllPlacementRuleBundles(suite.ctx) + suite.checkRule(re, testRule, 1, true) + ruleGroups, err := suite.client.GetAllPlacementRuleGroups(suite.ctx) re.NoError(err) - re.Len(bundles, 1) - re.Equal(placement.DefaultGroupID, bundles[0].ID) - re.Len(bundles[0].Rules, 1) - // Make sure the create timestamp is not zero to pass the later assertion. - testRule.CreateTimestamp = bundles[0].Rules[0].CreateTimestamp - re.Equal(testRule, bundles[0].Rules[0]) + re.Len(ruleGroups, 1) + re.Equal(placement.DefaultGroupID, ruleGroups[0].ID) + ruleGroup, err := suite.client.GetPlacementRuleGroupByID(suite.ctx, placement.DefaultGroupID) + re.NoError(err) + re.Equal(ruleGroups[0], ruleGroup) + testRuleGroup := &pd.RuleGroup{ + ID: "test-group", + Index: 1, + Override: true, + } + err = suite.client.SetPlacementRuleGroup(suite.ctx, testRuleGroup) + re.NoError(err) + ruleGroup, err = suite.client.GetPlacementRuleGroupByID(suite.ctx, testRuleGroup.ID) + re.NoError(err) + re.Equal(testRuleGroup, ruleGroup) + err = suite.client.DeletePlacementRuleGroupByID(suite.ctx, testRuleGroup.ID) + re.NoError(err) + ruleGroup, err = suite.client.GetPlacementRuleGroupByID(suite.ctx, testRuleGroup.ID) + re.ErrorContains(err, http.StatusText(http.StatusNotFound)) + re.Empty(ruleGroup) +} + +func (suite *httpClientTestSuite) checkRule( + re *require.Assertions, + rule *pd.Rule, totalRuleCount int, exist bool, +) { + // Check through the `GetPlacementRulesByGroup` API. + rules, err := suite.client.GetPlacementRulesByGroup(suite.ctx, rule.GroupID) + re.NoError(err) + checkRuleFunc(re, rules, rule, totalRuleCount, exist) + // Check through the `GetPlacementRuleBundleByGroup` API. + bundle, err := suite.client.GetPlacementRuleBundleByGroup(suite.ctx, rule.GroupID) + re.NoError(err) + checkRuleFunc(re, bundle.Rules, rule, totalRuleCount, exist) +} + +func checkRuleFunc( + re *require.Assertions, + rules []*pd.Rule, rule *pd.Rule, totalRuleCount int, exist bool, +) { + re.Len(rules, totalRuleCount) + for _, r := range rules { + if r.ID != rule.ID { + continue + } + re.Equal(rule.GroupID, r.GroupID) + re.Equal(rule.ID, r.ID) + re.Equal(rule.Role, r.Role) + re.Equal(rule.Count, r.Count) + return + } + if exist { + re.Failf("Failed to check the rule", "rule %+v not found", rule) + } } func (suite *httpClientTestSuite) TestRegionLabel() { @@ -202,10 +325,36 @@ func (suite *httpClientTestSuite) TestRegionLabel() { func (suite *httpClientTestSuite) TestAccelerateSchedule() { re := suite.Require() - suspectRegions := suite.cluster.GetLeaderServer().GetRaftCluster().GetSuspectRegions() + raftCluster := suite.cluster.GetLeaderServer().GetRaftCluster() + suspectRegions := raftCluster.GetSuspectRegions() re.Len(suspectRegions, 0) - err := suite.client.AccelerateSchedule(suite.ctx, []byte("a1"), []byte("a2")) + err := suite.client.AccelerateSchedule(suite.ctx, pd.NewKeyRange([]byte("a1"), []byte("a2"))) re.NoError(err) - suspectRegions = suite.cluster.GetLeaderServer().GetRaftCluster().GetSuspectRegions() + suspectRegions = raftCluster.GetSuspectRegions() re.Len(suspectRegions, 1) + raftCluster.ClearSuspectRegions() + suspectRegions = raftCluster.GetSuspectRegions() + re.Len(suspectRegions, 0) + err = suite.client.AccelerateScheduleInBatch(suite.ctx, []*pd.KeyRange{ + pd.NewKeyRange([]byte("a1"), []byte("a2")), + pd.NewKeyRange([]byte("a2"), []byte("a3")), + }) + re.NoError(err) + suspectRegions = raftCluster.GetSuspectRegions() + re.Len(suspectRegions, 2) +} + +func (suite *httpClientTestSuite) TestScheduleConfig() { + re := suite.Require() + config, err := suite.client.GetScheduleConfig(suite.ctx) + re.NoError(err) + re.Equal(float64(4), config["leader-schedule-limit"]) + re.Equal(float64(2048), config["region-schedule-limit"]) + config["leader-schedule-limit"] = float64(8) + err = suite.client.SetScheduleConfig(suite.ctx, config) + re.NoError(err) + config, err = suite.client.GetScheduleConfig(suite.ctx) + re.NoError(err) + re.Equal(float64(8), config["leader-schedule-limit"]) + re.Equal(float64(2048), config["region-schedule-limit"]) } diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index f6df0eb4de0..75d70e3cf06 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -15,7 +15,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/kvproto v0.0.0-20231018065736-c0689aded40c github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 - github.com/stretchr/testify v1.8.2 + github.com/stretchr/testify v1.8.3 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 @@ -50,9 +50,11 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect + github.com/bytedance/sonic v1.9.1 // indirect github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cenkalti/backoff/v4 v4.0.2 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect @@ -61,23 +63,24 @@ require ( github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 // indirect github.com/elliotchance/pie/v2 v2.1.0 // indirect github.com/fogleman/gg v1.3.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/gin-contrib/cors v1.4.0 // indirect github.com/gin-contrib/gzip v0.0.1 // indirect github.com/gin-contrib/pprof v1.4.0 // indirect github.com/gin-contrib/sse v0.1.0 // indirect - github.com/gin-gonic/gin v1.8.1 // indirect + github.com/gin-gonic/gin v1.9.1 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect github.com/go-openapi/jsonreference v0.19.6 // indirect github.com/go-openapi/spec v0.20.4 // indirect github.com/go-openapi/swag v0.19.15 // indirect - github.com/go-playground/locales v0.14.0 // indirect - github.com/go-playground/universal-translator v0.18.0 // indirect - github.com/go-playground/validator/v10 v10.10.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.0.9 // indirect - github.com/goccy/go-json v0.9.7 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect @@ -101,11 +104,12 @@ 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/cpuid/v2 v2.2.4 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/leodido/go-urn v1.2.1 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect github.com/mailru/easyjson v0.7.6 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/mattn/go-sqlite3 v1.14.15 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/minio/sio v0.3.0 // indirect @@ -113,13 +117,13 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pelletier/go-toml/v2 v2.0.1 // indirect + github.com/pelletier/go-toml/v2 v2.0.8 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -147,7 +151,8 @@ require ( github.com/tklauser/go-sysconf v0.3.11 // indirect github.com/tklauser/numcpus v0.6.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 // indirect - github.com/ugorji/go/codec v1.2.7 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/ugorji/go/codec v1.2.11 // indirect github.com/unrolled/render v1.0.1 // indirect github.com/urfave/negroni v0.3.0 // indirect github.com/vmihailenco/msgpack/v5 v5.3.5 // indirect @@ -159,6 +164,7 @@ require ( go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect + golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.14.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.5.0 // indirect diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index fc1dc1bbea5..dfead54afe1 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -68,6 +68,9 @@ github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4Yn github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpfpgJhvkuYWBmOYbo= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.0.2 h1:JIufpQLbh4DkbQoii76ItQIUFzevQSqOLZca4eamEDs= @@ -77,6 +80,9 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= 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= @@ -127,6 +133,8 @@ github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzP github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= +github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= +github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/cors v1.4.0 h1:oJ6gwtUl3lqV0WEIwM/LxPF1QZ5qe2lGWdY2+bz7y0g= github.com/gin-contrib/cors v1.4.0/go.mod h1:bs9pNM0x/UsmHPBWT2xZz9ROh8xYjYkiURUfmBoMlcs= @@ -139,8 +147,9 @@ github.com/gin-contrib/sse v0.1.0 h1:Y/yl/+YNO8GZSjAhjMsSuLt29uWRFHdHYUb5lYOV9qE github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= -github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= +github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= +github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -159,17 +168,21 @@ github.com/go-openapi/spec v0.20.4/go.mod h1:faYFR1CvsJZ0mNsmsphTMSoRrNV3TEDoAM7 github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.15 h1:D2NRCBzS9/pEY3gP9Nl8aDqGUcPFrwG2p+CNFrLyrCM= github.com/go-openapi/swag v0.19.15/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= -github.com/go-playground/assert/v2 v2.0.1 h1:MsBgLAaY856+nPRTKrp3/OZK38U/wa0CcBYNjji3q3A= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/locales v0.14.0 h1:u50s323jtVGugKlcYeyzC0etD1HifMjqmJqb8WugfUU= github.com/go-playground/locales v0.14.0/go.mod h1:sawfccIbzZTqEDETgFXqTho0QybSa7l++s0DH+LDiLs= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= -github.com/go-playground/universal-translator v0.18.0 h1:82dyy6p4OuJq4/CByFNOn/jYrnRPArHwAcmLoJZxyho= github.com/go-playground/universal-translator v0.18.0/go.mod h1:UvRDBj+xPUEGrFYl+lu/H90nyDXpg0fqeB/AQUGNTVA= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= -github.com/go-playground/validator/v10 v10.10.0 h1:I7mrTYv78z8k8VXa/qJlOlEXn/nBh+BF8dHX5nt/dr0= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= +github.com/go-playground/validator/v10 v10.14.0 h1:vgvQWe3XCz3gIeFDm/HnTIbj6UGmg/+t63MyGU2n5js= +github.com/go-playground/validator/v10 v10.14.0/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -177,8 +190,9 @@ github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9 github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-graphviz v0.0.9 h1:s/FMMJ1Joj6La3S5ApO3Jk2cwM4LpXECC2muFx3IPQQ= github.com/goccy/go-graphviz v0.0.9/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= -github.com/goccy/go-json v0.9.7 h1:IcB+Aqpx/iMHu5Yooh7jEzJk1JZ7Pjtmys2ukPr7EeM= github.com/goccy/go-json v0.9.7/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -315,6 +329,9 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= 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/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -328,8 +345,9 @@ 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/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= -github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= @@ -341,8 +359,9 @@ github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJ github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.14.15 h1:vfoHhTN1af61xCRSWzFIWzx2YskyMTwHLrExkBOjvxI= github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= @@ -389,8 +408,9 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml/v2 v2.0.1 h1:8e3L2cCQzLFi2CR4g7vGFuFxX7Jl1kKX8gW+iV0GUKU= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= +github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -414,8 +434,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 h1:wnHt7ETIB0vm+gbLx8QhcIEmRtrT4QlWlfpcI9vjxOk= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537/go.mod h1:EZ90+V5S4TttbYag6oKZ3jcNKRwZe1Mc9vXwOt9JBYw= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e h1:SJUSDejvKtj9vSh5ptRHh4iMrvPV3oKO8yp6/SYE8vc= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -512,8 +532,10 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ 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.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2/go.mod h1:lKJPbtWzJ9JhsTN1k1gZgleJWY/cqq0psdoMmaThG3w= github.com/swaggo/http-swagger v1.2.6 h1:ihTjChUoSRMpFMjWw+0AkL1Ti4r6v8pCgVYLmQVRlRw= @@ -539,13 +561,16 @@ github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go v1.2.7/go.mod h1:nF9osbDWLy6bDVv/Rtoh6QgnvNDpmCalQV5urGCCS6M= github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= +github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= +github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= @@ -603,6 +628,9 @@ go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -712,6 +740,7 @@ golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/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-20220704084225-05e143d24a9e/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.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -844,6 +873,7 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= moul.io/zapgorm2 v1.1.0 h1:qwAlMBYf+qJkJ7PAzJl4oCe6eS6QGiKAXUPeis0+RBE= moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index 1415acc46d1..7dcce498d56 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -301,7 +301,10 @@ func (suite *keyspaceGroupTestSuite) TestDefaultKeyspaceGroup() { return code == http.StatusOK && kg != nil }, testutil.WithWaitFor(time.Second*1)) suite.Equal(utils.DefaultKeyspaceGroupID, kg.ID) - suite.Len(kg.Members, utils.DefaultKeyspaceGroupReplicaCount) + // the allocNodesToAllKeyspaceGroups loop will run every 100ms. + testutil.Eventually(suite.Require(), func() bool { + return len(kg.Members) == utils.DefaultKeyspaceGroupReplicaCount + }) for _, member := range kg.Members { suite.Contains(nodes, member.Address) } diff --git a/tests/integrations/mcs/scheduling/config_test.go b/tests/integrations/mcs/scheduling/config_test.go index ccf7cdaf48c..69d77bb24ac 100644 --- a/tests/integrations/mcs/scheduling/config_test.go +++ b/tests/integrations/mcs/scheduling/config_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/cache" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/mcs/scheduling/server/config" sc "github.com/tikv/pd/pkg/schedule/config" @@ -84,7 +85,7 @@ func (suite *configTestSuite) TestConfigWatch() { suite.ctx, suite.pdLeaderServer.GetEtcdClient(), suite.cluster.GetCluster().GetId(), - config.NewPersistConfig(config.NewConfig()), + config.NewPersistConfig(config.NewConfig(), cache.NewStringTTL(suite.ctx, sc.DefaultGCInterval, sc.DefaultTTL)), endpoint.NewStorageEndpoint(kv.NewMemoryKV(), nil), ) re.NoError(err) @@ -144,7 +145,7 @@ func (suite *configTestSuite) TestSchedulerConfigWatch() { suite.ctx, suite.pdLeaderServer.GetEtcdClient(), suite.cluster.GetCluster().GetId(), - config.NewPersistConfig(config.NewConfig()), + config.NewPersistConfig(config.NewConfig(), cache.NewStringTTL(suite.ctx, sc.DefaultGCInterval, sc.DefaultTTL)), storage, ) re.NoError(err) diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index 7e833943e6e..309ea9dbc4d 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -47,9 +47,11 @@ require ( github.com/beorn7/perks v1.0.1 // indirect github.com/bitly/go-simplejson v0.5.0 // indirect github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch // indirect + github.com/bytedance/sonic v1.9.1 // indirect github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cenkalti/backoff/v4 v4.0.2 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect @@ -58,23 +60,24 @@ require ( github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 // indirect github.com/elliotchance/pie/v2 v2.1.0 // indirect github.com/fogleman/gg v1.3.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/gin-contrib/cors v1.4.0 // indirect github.com/gin-contrib/gzip v0.0.1 // indirect github.com/gin-contrib/pprof v1.4.0 // indirect github.com/gin-contrib/sse v0.1.0 // indirect - github.com/gin-gonic/gin v1.8.1 // indirect + github.com/gin-gonic/gin v1.9.1 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/go-openapi/jsonpointer v0.19.5 // indirect github.com/go-openapi/jsonreference v0.19.6 // indirect github.com/go-openapi/spec v0.20.4 // indirect github.com/go-openapi/swag v0.19.15 // indirect - github.com/go-playground/locales v0.14.0 // indirect - github.com/go-playground/universal-translator v0.18.0 // indirect - github.com/go-playground/validator/v10 v10.10.0 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.0 // indirect github.com/go-resty/resty/v2 v2.6.0 // indirect github.com/go-sql-driver/mysql v1.7.0 // indirect github.com/goccy/go-graphviz v0.0.9 // indirect - github.com/goccy/go-json v0.9.7 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect @@ -98,11 +101,12 @@ 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/cpuid/v2 v2.2.4 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/leodido/go-urn v1.2.1 // indirect + github.com/leodido/go-urn v1.2.4 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect github.com/mailru/easyjson v0.7.6 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/mattn/go-sqlite3 v1.14.15 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/minio/sio v0.3.0 // indirect @@ -110,14 +114,14 @@ require ( github.com/modern-go/reflect2 v1.0.2 // indirect github.com/oleiade/reflections v1.0.1 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pelletier/go-toml/v2 v2.0.1 // indirect + github.com/pelletier/go-toml/v2 v2.0.8 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -145,7 +149,8 @@ require ( github.com/tklauser/go-sysconf v0.3.11 // indirect github.com/tklauser/numcpus v0.6.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 // indirect - github.com/ugorji/go/codec v1.2.7 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/ugorji/go/codec v1.2.11 // indirect github.com/unrolled/render v1.0.1 // indirect github.com/urfave/negroni v0.3.0 // indirect github.com/vmihailenco/msgpack/v5 v5.3.5 // indirect @@ -160,6 +165,7 @@ require ( go.uber.org/goleak v1.1.12 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.24.0 // indirect + golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.14.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/image v0.5.0 // indirect diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index 65a7f3e3558..94fbde2ad57 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -68,6 +68,9 @@ github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4Yn github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch h1:KLE/YeX+9FNaGVW5MtImRVPhjDpfpgJhvkuYWBmOYbo= github.com/breeswish/gin-jwt/v2 v2.6.4-jwt-patch/go.mod h1:KjBLriHXe7L6fGceqWzTod8HUB/TP1WWDtfuSYtYXaI= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/cenkalti/backoff/v4 v4.0.2 h1:JIufpQLbh4DkbQoii76ItQIUFzevQSqOLZca4eamEDs= @@ -77,6 +80,9 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= 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= @@ -123,6 +129,8 @@ github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5Kwzbycv github.com/fogleman/gg v1.3.0 h1:/7zJX8F6AaYQc57WQCyN9cAIz+4bCJGO9B+dyW29am8= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= +github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/cors v1.4.0 h1:oJ6gwtUl3lqV0WEIwM/LxPF1QZ5qe2lGWdY2+bz7y0g= github.com/gin-contrib/cors v1.4.0/go.mod h1:bs9pNM0x/UsmHPBWT2xZz9ROh8xYjYkiURUfmBoMlcs= @@ -135,8 +143,9 @@ github.com/gin-contrib/sse v0.1.0 h1:Y/yl/+YNO8GZSjAhjMsSuLt29uWRFHdHYUb5lYOV9qE github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= -github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= +github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= +github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -155,17 +164,21 @@ github.com/go-openapi/spec v0.20.4/go.mod h1:faYFR1CvsJZ0mNsmsphTMSoRrNV3TEDoAM7 github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.15 h1:D2NRCBzS9/pEY3gP9Nl8aDqGUcPFrwG2p+CNFrLyrCM= github.com/go-openapi/swag v0.19.15/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= -github.com/go-playground/assert/v2 v2.0.1 h1:MsBgLAaY856+nPRTKrp3/OZK38U/wa0CcBYNjji3q3A= github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= -github.com/go-playground/locales v0.14.0 h1:u50s323jtVGugKlcYeyzC0etD1HifMjqmJqb8WugfUU= github.com/go-playground/locales v0.14.0/go.mod h1:sawfccIbzZTqEDETgFXqTho0QybSa7l++s0DH+LDiLs= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= -github.com/go-playground/universal-translator v0.18.0 h1:82dyy6p4OuJq4/CByFNOn/jYrnRPArHwAcmLoJZxyho= github.com/go-playground/universal-translator v0.18.0/go.mod h1:UvRDBj+xPUEGrFYl+lu/H90nyDXpg0fqeB/AQUGNTVA= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= github.com/go-playground/validator/v10 v10.2.0/go.mod h1:uOYAAleCW8F/7oMFd6aG0GOhaH6EGOAJShg8Id5JGkI= -github.com/go-playground/validator/v10 v10.10.0 h1:I7mrTYv78z8k8VXa/qJlOlEXn/nBh+BF8dHX5nt/dr0= github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= +github.com/go-playground/validator/v10 v10.14.0 h1:vgvQWe3XCz3gIeFDm/HnTIbj6UGmg/+t63MyGU2n5js= +github.com/go-playground/validator/v10 v10.14.0/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-resty/resty/v2 v2.6.0 h1:joIR5PNLM2EFqqESUjCMGXrWmXNHEU9CEiK813oKYS4= github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.7.0 h1:ueSltNNllEqE3qcWBTD0iQd3IpL/6U+mJxLkazJ7YPc= @@ -173,8 +186,9 @@ github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9 github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/goccy/go-graphviz v0.0.9 h1:s/FMMJ1Joj6La3S5ApO3Jk2cwM4LpXECC2muFx3IPQQ= github.com/goccy/go-graphviz v0.0.9/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= -github.com/goccy/go-json v0.9.7 h1:IcB+Aqpx/iMHu5Yooh7jEzJk1JZ7Pjtmys2ukPr7EeM= github.com/goccy/go-json v0.9.7/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -312,6 +326,9 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= 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/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -325,8 +342,9 @@ 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/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= -github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= @@ -338,8 +356,9 @@ github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJ github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-sqlite3 v1.14.15 h1:vfoHhTN1af61xCRSWzFIWzx2YskyMTwHLrExkBOjvxI= github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= @@ -383,8 +402,9 @@ github.com/otiai10/mint v1.3.0/go.mod h1:F5AjcsTsWUqX+Na9fpHb52P8pcRX2CI6A3ctIT9 github.com/otiai10/mint v1.3.3/go.mod h1:/yxELlJQ0ufhjUwhshSj+wFjZ78CnZ48/1wtmBH1OTc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml/v2 v2.0.1 h1:8e3L2cCQzLFi2CR4g7vGFuFxX7Jl1kKX8gW+iV0GUKU= github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= +github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -408,8 +428,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537 h1:wnHt7ETIB0vm+gbLx8QhcIEmRtrT4QlWlfpcI9vjxOk= -github.com/pingcap/tidb-dashboard v0.0.0-20231108071238-7cb8b7ff0537/go.mod h1:EZ90+V5S4TttbYag6oKZ3jcNKRwZe1Mc9vXwOt9JBYw= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e h1:SJUSDejvKtj9vSh5ptRHh4iMrvPV3oKO8yp6/SYE8vc= +github.com/pingcap/tidb-dashboard v0.0.0-20231127105651-ce4097837c5e/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -506,7 +526,9 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ 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.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 h1:+iNTcqQJy0OZ5jk6a5NLib47eqXK8uYcPX+O4+cBpEM= @@ -534,13 +556,16 @@ github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go v1.2.7/go.mod h1:nF9osbDWLy6bDVv/Rtoh6QgnvNDpmCalQV5urGCCS6M= github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= +github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= +github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= @@ -598,6 +623,9 @@ go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -707,6 +735,7 @@ golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/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-20220704084225-05e143d24a9e/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.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -840,6 +869,7 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= moul.io/zapgorm2 v1.1.0 h1:qwAlMBYf+qJkJ7PAzJl4oCe6eS6QGiKAXUPeis0+RBE= moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/tests/pdctl/config/config_test.go b/tests/pdctl/config/config_test.go index badccd9becc..3b3310185b1 100644 --- a/tests/pdctl/config/config_test.go +++ b/tests/pdctl/config/config_test.go @@ -24,6 +24,7 @@ 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" @@ -60,8 +61,10 @@ func TestConfigTestSuite(t *testing.T) { } func (suite *configTestSuite) TestConfig() { + suite.NoError(failpoint.Enable("github.com/tikv/pd/pkg/dashboard/adapter/skipDashboardLoop", `return(true)`)) env := tests.NewSchedulingTestEnvironment(suite.T()) env.RunTestInTwoModes(suite.checkConfig) + suite.NoError(failpoint.Disable("github.com/tikv/pd/pkg/dashboard/adapter/skipDashboardLoop")) } func (suite *configTestSuite) checkConfig(cluster *tests.TestCluster) { @@ -683,66 +686,66 @@ func (suite *configTestSuite) checkUpdateDefaultReplicaConfig(cluster *tests.Tes checkMaxReplicas := func(expect uint64) { args := []string{"-u", pdAddr, "config", "show", "replication"} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - replicationCfg := sc.ReplicationConfig{} - re.NoError(json.Unmarshal(output, &replicationCfg)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + replicationCfg := sc.ReplicationConfig{} + re.NoError(json.Unmarshal(output, &replicationCfg)) return replicationCfg.MaxReplicas == expect }) } checkLocationLabels := func(expect int) { args := []string{"-u", pdAddr, "config", "show", "replication"} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - replicationCfg := sc.ReplicationConfig{} - re.NoError(json.Unmarshal(output, &replicationCfg)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + replicationCfg := sc.ReplicationConfig{} + re.NoError(json.Unmarshal(output, &replicationCfg)) return len(replicationCfg.LocationLabels) == expect }) } checkIsolationLevel := func(expect string) { args := []string{"-u", pdAddr, "config", "show", "replication"} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - replicationCfg := sc.ReplicationConfig{} - re.NoError(json.Unmarshal(output, &replicationCfg)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + replicationCfg := sc.ReplicationConfig{} + re.NoError(json.Unmarshal(output, &replicationCfg)) return replicationCfg.IsolationLevel == expect }) } checkRuleCount := func(expect int) { args := []string{"-u", pdAddr, "config", "placement-rules", "show", "--group", placement.DefaultGroupID, "--id", placement.DefaultRuleID} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - rule := placement.Rule{} - re.NoError(json.Unmarshal(output, &rule)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + rule := placement.Rule{} + re.NoError(json.Unmarshal(output, &rule)) return rule.Count == expect }) } checkRuleLocationLabels := func(expect int) { args := []string{"-u", pdAddr, "config", "placement-rules", "show", "--group", placement.DefaultGroupID, "--id", placement.DefaultRuleID} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - rule := placement.Rule{} - re.NoError(json.Unmarshal(output, &rule)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + rule := placement.Rule{} + re.NoError(json.Unmarshal(output, &rule)) return len(rule.LocationLabels) == expect }) } checkRuleIsolationLevel := func(expect string) { args := []string{"-u", pdAddr, "config", "placement-rules", "show", "--group", placement.DefaultGroupID, "--id", placement.DefaultRuleID} - output, err := pdctl.ExecuteCommand(cmd, args...) - re.NoError(err) - rule := placement.Rule{} - re.NoError(json.Unmarshal(output, &rule)) testutil.Eventually(re, func() bool { // wait for the config to be synced to the scheduling server + output, err := pdctl.ExecuteCommand(cmd, args...) + re.NoError(err) + rule := placement.Rule{} + re.NoError(json.Unmarshal(output, &rule)) return rule.IsolationLevel == expect }) } diff --git a/tests/pdctl/keyspace/keyspace_test.go b/tests/pdctl/keyspace/keyspace_test.go index 805a30e6f18..3ff755fe601 100644 --- a/tests/pdctl/keyspace/keyspace_test.go +++ b/tests/pdctl/keyspace/keyspace_test.go @@ -105,6 +105,35 @@ func TestKeyspace(t *testing.T) { re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) } +// Show command should auto retry without refresh_group_id if keyspace group manager not initialized. +// See issue: #7441 +func TestKeyspaceGroupUninitialized(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + 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 := tests.NewTestCluster(ctx, 1) + re.NoError(err) + re.NoError(tc.RunInitialServers()) + tc.WaitLeader() + re.NoError(tc.GetLeaderServer().BootstrapCluster()) + pdAddr := tc.GetConfig().GetClientURL() + + keyspaceName := "DEFAULT" + keyspaceID := uint32(0) + args := []string{"-u", pdAddr, "keyspace", "show", "name", keyspaceName} + output, err := pdctl.ExecuteCommand(pdctlCmd.GetRootCmd(), args...) + re.NoError(err) + var meta api.KeyspaceMeta + re.NoError(json.Unmarshal(output, &meta)) + re.Equal(keyspaceName, meta.GetName()) + re.Equal(keyspaceID, meta.GetId()) + + re.NoError(failpoint.Disable("github.com/tikv/pd/server/delayStartServerLoop")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/skipSplitRegion")) +} + type keyspaceTestSuite struct { suite.Suite ctx context.Context diff --git a/tests/server/config/config_test.go b/tests/server/config/config_test.go index 4a4a91f2661..c0e9a0e1148 100644 --- a/tests/server/config/config_test.go +++ b/tests/server/config/config_test.go @@ -448,7 +448,7 @@ func (suite *configTestSuite) assertTTLConfig( if !expectedEqual { equality = suite.NotEqual } - checkfunc := func(options ttlConfigInterface) { + checkFunc := func(options ttlConfigInterface) { equality(uint64(999), options.GetMaxSnapshotCount()) equality(false, options.IsLocationReplacementEnabled()) equality(uint64(999), options.GetMaxMergeRegionSize()) @@ -461,7 +461,7 @@ func (suite *configTestSuite) assertTTLConfig( equality(uint64(999), options.GetMergeScheduleLimit()) equality(false, options.IsTikvRegionSplitEnabled()) } - checkfunc(cluster.GetLeaderServer().GetServer().GetPersistOptions()) + checkFunc(cluster.GetLeaderServer().GetServer().GetPersistOptions()) if cluster.GetSchedulingPrimaryServer() != nil { // wait for the scheduling primary server to be synced options := cluster.GetSchedulingPrimaryServer().GetPersistConfig() @@ -471,16 +471,16 @@ func (suite *configTestSuite) assertTTLConfig( } return uint64(999) != options.GetMaxSnapshotCount() }) - checkfunc(options) + checkFunc(options) } } -func (suite *configTestSuite) assertTTLConfigItemEqaul( +func (suite *configTestSuite) assertTTLConfigItemEqual( cluster *tests.TestCluster, item string, expectedValue interface{}, ) { - checkfunc := func(options ttlConfigInterface) bool { + checkFunc := func(options ttlConfigInterface) bool { switch item { case "max-merge-region-size": return expectedValue.(uint64) == options.GetMaxMergeRegionSize() @@ -491,11 +491,11 @@ func (suite *configTestSuite) assertTTLConfigItemEqaul( } return false } - suite.True(checkfunc(cluster.GetLeaderServer().GetServer().GetPersistOptions())) + suite.True(checkFunc(cluster.GetLeaderServer().GetServer().GetPersistOptions())) if cluster.GetSchedulingPrimaryServer() != nil { // wait for the scheduling primary server to be synced tu.Eventually(suite.Require(), func() bool { - return checkfunc(cluster.GetSchedulingPrimaryServer().GetPersistConfig()) + return checkFunc(cluster.GetSchedulingPrimaryServer().GetPersistConfig()) }) } } @@ -506,8 +506,7 @@ func createTTLUrl(url string, ttl int) string { func (suite *configTestSuite) TestConfigTTL() { env := tests.NewSchedulingTestEnvironment(suite.T()) - // FIXME: enable this test in two modes after ttl config is supported. - env.RunTestInPDMode(suite.checkConfigTTL) + env.RunTestInTwoModes(suite.checkConfigTTL) } func (suite *configTestSuite) checkConfigTTL(cluster *tests.TestCluster) { @@ -523,14 +522,14 @@ func (suite *configTestSuite) checkConfigTTL(cluster *tests.TestCluster) { suite.assertTTLConfig(cluster, false) // test time goes by - err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 1), postData, tu.StatusOK(re)) + err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 5), postData, tu.StatusOK(re)) suite.NoError(err) suite.assertTTLConfig(cluster, true) - time.Sleep(2 * time.Second) + time.Sleep(5 * time.Second) suite.assertTTLConfig(cluster, false) // test cleaning up - err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 1), postData, tu.StatusOK(re)) + err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 5), postData, tu.StatusOK(re)) suite.NoError(err) suite.assertTTLConfig(cluster, true) err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 0), postData, tu.StatusOK(re)) @@ -552,9 +551,9 @@ func (suite *configTestSuite) checkConfigTTL(cluster *tests.TestCluster) { err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 1), postData, tu.StatusOK(re)) suite.NoError(err) - suite.assertTTLConfigItemEqaul(cluster, "max-merge-region-size", uint64(999)) + suite.assertTTLConfigItemEqual(cluster, "max-merge-region-size", uint64(999)) // max-merge-region-keys should keep consistence with max-merge-region-size. - suite.assertTTLConfigItemEqaul(cluster, "max-merge-region-keys", uint64(999*10000)) + suite.assertTTLConfigItemEqual(cluster, "max-merge-region-keys", uint64(999*10000)) // on invalid value, we use default config mergeConfig = map[string]interface{}{ @@ -564,13 +563,12 @@ func (suite *configTestSuite) checkConfigTTL(cluster *tests.TestCluster) { suite.NoError(err) err = tu.CheckPostJSON(testDialClient, createTTLUrl(urlPrefix, 10), postData, tu.StatusOK(re)) suite.NoError(err) - suite.assertTTLConfigItemEqaul(cluster, "enable-tikv-split-region", true) + suite.assertTTLConfigItemEqual(cluster, "enable-tikv-split-region", true) } func (suite *configTestSuite) TestTTLConflict() { env := tests.NewSchedulingTestEnvironment(suite.T()) - // FIXME: enable this test in two modes after ttl config is supported. - env.RunTestInPDMode(suite.checkTTLConflict) + env.RunTestInTwoModes(suite.checkTTLConflict) } func (suite *configTestSuite) checkTTLConflict(cluster *tests.TestCluster) { diff --git a/tools/pd-api-bench/go.mod b/tools/pd-api-bench/go.mod index e6e896a0797..8050f433e8b 100644 --- a/tools/pd-api-bench/go.mod +++ b/tools/pd-api-bench/go.mod @@ -26,8 +26,10 @@ require ( github.com/aws/smithy-go v1.13.5 // indirect github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect + github.com/bytedance/sonic v1.9.1 // indirect github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect @@ -35,12 +37,13 @@ require ( github.com/docker/go-units v0.4.0 // indirect github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 // indirect github.com/elliotchance/pie/v2 v2.1.0 // indirect + github.com/gabriel-vasile/mimetype v1.4.2 // indirect github.com/gin-contrib/sse v0.1.0 // indirect - github.com/gin-gonic/gin v1.8.1 // indirect - github.com/go-playground/locales v0.14.0 // indirect - github.com/go-playground/universal-translator v0.18.0 // indirect - github.com/go-playground/validator/v10 v10.10.0 // indirect - github.com/goccy/go-json v0.9.7 // indirect + github.com/gin-gonic/gin v1.9.1 // indirect + github.com/go-playground/locales v0.14.1 // indirect + github.com/go-playground/universal-translator v0.18.1 // indirect + github.com/go-playground/validator/v10 v10.14.0 // indirect + github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef // indirect @@ -56,14 +59,15 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/jonboulle/clockwork v0.2.2 // indirect github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/konsorten/go-windows-terminal-sequences v1.0.3 // indirect - github.com/leodido/go-urn v1.2.1 // indirect - github.com/mattn/go-isatty v0.0.14 // indirect + github.com/leodido/go-urn v1.2.4 // indirect + github.com/mattn/go-isatty v0.0.19 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pelletier/go-toml/v2 v2.0.1 // indirect + github.com/pelletier/go-toml/v2 v2.0.8 // indirect github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 // indirect github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d // indirect github.com/pingcap/errcode v0.3.0 // indirect @@ -81,10 +85,11 @@ require ( github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 // indirect github.com/soheilhy/cmux v0.1.4 // indirect github.com/spf13/pflag v1.0.5 // indirect - github.com/stretchr/testify v1.8.2 // indirect + github.com/stretchr/testify v1.8.3 // indirect github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 // indirect - github.com/ugorji/go/codec v1.2.7 // indirect + github.com/twitchyliquid64/golang-asm v0.15.1 // indirect + github.com/ugorji/go/codec v1.2.11 // indirect github.com/unrolled/render v1.0.1 // indirect github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect go.etcd.io/bbolt v1.3.6 // indirect @@ -92,6 +97,7 @@ require ( go.uber.org/atomic v1.10.0 // indirect go.uber.org/goleak v1.1.12 // indirect go.uber.org/multierr v1.11.0 // indirect + golang.org/x/arch v0.3.0 // indirect golang.org/x/crypto v0.14.0 // indirect golang.org/x/exp v0.0.0-20230711005742-c3f37128e5a4 // indirect golang.org/x/net v0.17.0 // indirect diff --git a/tools/pd-api-bench/go.sum b/tools/pd-api-bench/go.sum index f40a4fe2f5a..1e40c511586 100644 --- a/tools/pd-api-bench/go.sum +++ b/tools/pd-api-bench/go.sum @@ -42,12 +42,18 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce 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/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bytedance/sonic v1.5.0/go.mod h1:ED5hyg4y6t3/9Ku1R6dU/4KyJ48DZ4jPhfY1O2AihPM= +github.com/bytedance/sonic v1.9.1 h1:6iJ6NqdoxCDr6mbY8h18oSO+cShGSMRGCEo7F2h0x8s= +github.com/bytedance/sonic v1.9.1/go.mod h1:i736AoUSYt75HyZLoJW9ERYxcy6eaN6h4BZXU064P/U= 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/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chenzhuoyu/base64x v0.0.0-20211019084208-fb5309c8db06/go.mod h1:DH46F32mSOjUmXrMHnKwZdA8wcEefY7UVqBKYGjpdQY= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311 h1:qSGYFH7+jGhDF8vLC+iwCD4WpbV1EBDSzWkJODFLams= +github.com/chenzhuoyu/base64x v0.0.0-20221115062448-fe3a3abad311/go.mod h1:b583jCggY9gE99b6G5LEC39OIiVsWj+R97kbl5odCEk= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= @@ -61,7 +67,6 @@ github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7 github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -80,11 +85,13 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/gabriel-vasile/mimetype v1.4.2 h1:w5qFW6JKBz9Y393Y4q372O9A7cUSequkh1Q7OhCmWKU= +github.com/gabriel-vasile/mimetype v1.4.2/go.mod h1:zApsH/mKG4w07erKIaJPFiX0Tsq9BFQgN3qGY5GnNgA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.1.0 h1:Y/yl/+YNO8GZSjAhjMsSuLt29uWRFHdHYUb5lYOV9qE= github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= -github.com/gin-gonic/gin v1.8.1 h1:4+fr/el88TOO3ewCmQr8cx/CtZ/umlIRIs5M4NTNjf8= -github.com/gin-gonic/gin v1.8.1/go.mod h1:ji8BvRH1azfM+SYow9zQ6SZMvR8qOMZHmsCuWR9tTTk= +github.com/gin-gonic/gin v1.9.1 h1:4idEAncQnU5cB7BeOkPtxjfCSye0AAm1R0RVIqJ+Jmg= +github.com/gin-gonic/gin v1.9.1/go.mod h1:hPrL7YrpYKXt5YId3A/Tnip5kqbEAP+KLuI3SUcPTeU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -93,17 +100,17 @@ github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-playground/assert/v2 v2.0.1 h1:MsBgLAaY856+nPRTKrp3/OZK38U/wa0CcBYNjji3q3A= -github.com/go-playground/assert/v2 v2.0.1/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= -github.com/go-playground/locales v0.14.0 h1:u50s323jtVGugKlcYeyzC0etD1HifMjqmJqb8WugfUU= -github.com/go-playground/locales v0.14.0/go.mod h1:sawfccIbzZTqEDETgFXqTho0QybSa7l++s0DH+LDiLs= -github.com/go-playground/universal-translator v0.18.0 h1:82dyy6p4OuJq4/CByFNOn/jYrnRPArHwAcmLoJZxyho= -github.com/go-playground/universal-translator v0.18.0/go.mod h1:UvRDBj+xPUEGrFYl+lu/H90nyDXpg0fqeB/AQUGNTVA= -github.com/go-playground/validator/v10 v10.10.0 h1:I7mrTYv78z8k8VXa/qJlOlEXn/nBh+BF8dHX5nt/dr0= -github.com/go-playground/validator/v10 v10.10.0/go.mod h1:74x4gJWsvQexRdW8Pn3dXSGrTK4nAUsbPlLADvpJkos= +github.com/go-playground/assert/v2 v2.2.0 h1:JvknZsQTYeFEAhQwI4qEt9cyV5ONwRHC+lYKSsYSR8s= +github.com/go-playground/assert/v2 v2.2.0/go.mod h1:VDjEfimB/XKnb+ZQfWdccd7VUvScMdVu0Titje2rxJ4= +github.com/go-playground/locales v0.14.1 h1:EWaQ/wswjilfKLTECiXz7Rh+3BjFhfDFKv/oXslEjJA= +github.com/go-playground/locales v0.14.1/go.mod h1:hxrqLVvrK65+Rwrd5Fc6F2O76J/NuW9t0sjnWqG1slY= +github.com/go-playground/universal-translator v0.18.1 h1:Bcnm0ZwsGyWbCzImXv+pAJnYK9S473LQFuzCbDbfSFY= +github.com/go-playground/universal-translator v0.18.1/go.mod h1:xekY+UJKNuX9WP91TpwSH2VMlDf28Uj24BCp08ZFTUY= +github.com/go-playground/validator/v10 v10.14.0 h1:vgvQWe3XCz3gIeFDm/HnTIbj6UGmg/+t63MyGU2n5js= +github.com/go-playground/validator/v10 v10.14.0/go.mod h1:9iXMNT7sEkjXb0I+enO7QXmzG6QCsPWY4zveKFVRSyU= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/goccy/go-json v0.9.7 h1:IcB+Aqpx/iMHu5Yooh7jEzJk1JZ7Pjtmys2ukPr7EeM= -github.com/goccy/go-json v0.9.7/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= +github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= +github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= @@ -183,26 +190,26 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= 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/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= +github.com/klauspost/cpuid/v2 v2.2.4 h1:acbojRNwl3o09bUq+yDCtZFc1aiwaAAxtcn8YkZXnvk= +github.com/klauspost/cpuid/v2 v2.2.4/go.mod h1:RVVoqg1df56z8g3pUjL/3lE5UfnlrJX8tyFgg4nqhuY= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.3 h1:CE8S1cTafDpPvMhIxNJKvHsGVBgn1xWYf1NbHQhywc8= github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= 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/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= -github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= +github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= +github.com/leodido/go-urn v1.2.4/go.mod h1:7ZrI8mTSeBSHl/UaRyKQW1qZeMgak41ANeCNaVckg+4= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= -github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.19 h1:JITubQf0MOLdlGRuRq+jtsDlekdYPia9ZFsB8h/APPA= +github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -224,8 +231,8 @@ github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= 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/pelletier/go-toml/v2 v2.0.1 h1:8e3L2cCQzLFi2CR4g7vGFuFxX7Jl1kKX8gW+iV0GUKU= -github.com/pelletier/go-toml/v2 v2.0.1/go.mod h1:r9LEWfGN8R5k0VXJ+0BkIe7MYkRdwZOjgMj2KwnJFUo= +github.com/pelletier/go-toml/v2 v2.0.8 h1:0ctb6s9mE31h0/lhu+J6OPmVeDxJn+kYnJc2jZR9tGQ= +github.com/pelletier/go-toml/v2 v2.0.8/go.mod h1:vuYfssBdrU2XDZ9bYydBu6t+6a6PYNcZljzZR9VXg+4= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= @@ -244,7 +251,6 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -274,9 +280,6 @@ github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3x github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= -github.com/rogpeppe/go-internal v1.8.0 h1:FCbCCtXNOY3UtUuHUYaghJg4y7Fd14rXifAYUAtL9R8= -github.com/rogpeppe/go-internal v1.8.0/go.mod h1:WmiCO8CzOY8rg0OYDC4/i/2WRWAB6poM+XZ2dLUbcbE= github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= @@ -305,12 +308,13 @@ github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpE github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= 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.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.3 h1:RP3t2pwF7cMEbC1dqtB6poj3niw/9gnV4Cjg5oW5gtY= +github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 h1:1oFLiOyVl+W7bnBzGhf7BbIv9loSFQcieWWYIjLqcAw= github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/tklauser/go-sysconf v0.3.11 h1:89WgdJhk5SNwJfu+GKyYveZ4IaJ7xAkecBo+KdJV0CM= @@ -319,9 +323,10 @@ github.com/tklauser/numcpus v0.6.0 h1:kebhY2Qt+3U6RNK7UqpYNA+tJ23IBEGKkB7JQBfDYm github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ5UVIcaL4= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/ugorji/go v1.2.7/go.mod h1:nF9osbDWLy6bDVv/Rtoh6QgnvNDpmCalQV5urGCCS6M= -github.com/ugorji/go/codec v1.2.7 h1:YPXUKf7fYbp/y8xloBqZOw2qaVggbfwMlI8WM3wZUJ0= -github.com/ugorji/go/codec v1.2.7/go.mod h1:WGN1fab3R1fzQlVQTkfxVtIBhWDRqOviHU95kRgeqEY= +github.com/twitchyliquid64/golang-asm v0.15.1 h1:SU5vSMR7hnwNxj24w34ZyCi/FmDZTkS4MhqMhdFk5YI= +github.com/twitchyliquid64/golang-asm v0.15.1/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= +github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4dU= +github.com/ugorji/go/codec v1.2.11/go.mod h1:UNopzCgEMSXjBc6AOMqYvWC1ktqTAfzJZUZgYf6w6lg= github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= @@ -354,11 +359,13 @@ go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.24.0 h1:FiJd5l1UOLj0wCgbSE0rwwXHzEdAZS6hiiSnxJN/D60= go.uber.org/zap v1.24.0/go.mod h1:2kMP+WWQ8aoFoedH3T2sq6iJ2yDWpHbP0f6MQbS9Gkg= +golang.org/x/arch v0.0.0-20210923205945-b76863e36670/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= +golang.org/x/arch v0.3.0 h1:02VY4/ZcO/gBOH6PUaoiptASxtXU10jazRCP865E97k= +golang.org/x/arch v0.3.0/go.mod h1:5om86z9Hs0C8fWVUuoMHwpExlXzs5Tkyp9hOrfG7pp8= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= 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-20220411220226-7b82a4e95df4/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= @@ -388,7 +395,6 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= @@ -425,9 +431,9 @@ golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220704084225-05e143d24a9e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -486,11 +492,9 @@ google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= 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 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= 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/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= @@ -515,6 +519,7 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/tools/pd-ctl/pdctl/command/keyspace_command.go b/tools/pd-ctl/pdctl/command/keyspace_command.go index 7c0d3d78bf6..93a99abc39f 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_command.go @@ -28,11 +28,12 @@ import ( const ( keyspacePrefix = "pd/api/v2/keyspaces" // flags - nmConfig = "config" - nmLimit = "limit" - nmPageToken = "page_token" - nmRemove = "remove" - nmUpdate = "update" + nmConfig = "config" + nmLimit = "limit" + nmPageToken = "page_token" + nmRemove = "remove" + nmUpdate = "update" + nmForceRefreshGroupID = "force_refresh_group_id" ) // NewKeyspaceCommand returns a keyspace subcommand of rootCmd. @@ -64,6 +65,7 @@ func newShowKeyspaceCommand() *cobra.Command { Short: "show keyspace metadata specified by keyspace name", Run: showKeyspaceNameCommandFunc, } + showByName.Flags().Bool(nmForceRefreshGroupID, true, "force refresh keyspace group id") r.AddCommand(showByID) r.AddCommand(showByName) return r @@ -87,7 +89,21 @@ func showKeyspaceNameCommandFunc(cmd *cobra.Command, args []string) { cmd.Usage() return } - resp, err := doRequest(cmd, fmt.Sprintf("%s/%s?force_refresh_group_id=true", keyspacePrefix, args[0]), http.MethodGet, http.Header{}) + refreshGroupID, err := cmd.Flags().GetBool(nmForceRefreshGroupID) + if err != nil { + cmd.PrintErrln("Failed to parse flag: ", err) + return + } + url := fmt.Sprintf("%s/%s", keyspacePrefix, args[0]) + if refreshGroupID { + url += "?force_refresh_group_id=true" + } + resp, err := doRequest(cmd, url, http.MethodGet, http.Header{}) + // Retry without the force_refresh_group_id if the keyspace group manager is not initialized. + // This can happen when PD is not running in API mode. + if err != nil && refreshGroupID && strings.Contains(err.Error(), handlers.GroupManagerUninitializedErr) { + resp, err = doRequest(cmd, fmt.Sprintf("%s/%s", keyspacePrefix, args[0]), http.MethodGet, http.Header{}) + } if err != nil { cmd.PrintErrln("Failed to get the keyspace information: ", err) return