From 3f67cacf5fd537281bcf8ebc6923a042ee012c19 Mon Sep 17 00:00:00 2001 From: Smilencer Date: Tue, 5 Mar 2024 11:47:04 +0800 Subject: [PATCH 01/10] client/http: fix the scan region parameter (#7870) close tikv/pd#7869 add null pointer check and fix http request format Signed-off-by: Smityz --- client/http/api.go | 2 +- client/pd_service_discovery.go | 4 +++- tests/integrations/client/http_client_test.go | 8 ++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/client/http/api.go b/client/http/api.go index 398307ec1ba..434bf0d1fe4 100644 --- a/client/http/api.go +++ b/client/http/api.go @@ -95,7 +95,7 @@ func RegionByKey(key []byte) 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", + return fmt.Sprintf("%s?key=%s&end_key=%s&limit=%d", regionsByKey, startKeyStr, endKeyStr, limit) } diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index 2c08f3ae833..137a5c640c0 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -893,7 +893,9 @@ func (c *pdServiceDiscovery) checkServiceModeChanged() error { // If the method is not supported, we set it to pd mode. // TODO: it's a hack way to solve the compatibility issue. // we need to remove this after all maintained version supports the method. - c.serviceModeUpdateCb(pdpb.ServiceMode_PD_SVC_MODE) + if c.serviceModeUpdateCb != nil { + c.serviceModeUpdateCb(pdpb.ServiceMode_PD_SVC_MODE) + } return nil } return err diff --git a/tests/integrations/client/http_client_test.go b/tests/integrations/client/http_client_test.go index 4fe6b955c5d..9bcc7bc9fbd 100644 --- a/tests/integrations/client/http_client_test.go +++ b/tests/integrations/client/http_client_test.go @@ -204,6 +204,14 @@ func (suite *httpClientTestSuite) checkMeta(mode mode, client pd.Client) { version, err := client.GetClusterVersion(env.ctx) re.NoError(err) re.Equal("0.0.0", version) + rgs, _ := client.GetRegionsByKeyRange(env.ctx, pd.NewKeyRange([]byte("a"), []byte("a1")), 100) + re.Equal(int64(0), rgs.Count) + rgs, _ = client.GetRegionsByKeyRange(env.ctx, pd.NewKeyRange([]byte("a1"), []byte("a3")), 100) + re.Equal(int64(2), rgs.Count) + rgs, _ = client.GetRegionsByKeyRange(env.ctx, pd.NewKeyRange([]byte("a2"), []byte("b")), 100) + re.Equal(int64(1), rgs.Count) + rgs, _ = client.GetRegionsByKeyRange(env.ctx, pd.NewKeyRange([]byte(""), []byte("")), 100) + re.Equal(int64(2), rgs.Count) } func (suite *httpClientTestSuite) TestGetMinResolvedTSByStoresIDs() { From 41012b5ad6012847bd2330f371f61e2bbf8ff8fb Mon Sep 17 00:00:00 2001 From: Yongbo Jiang Date: Tue, 5 Mar 2024 15:29:07 +0800 Subject: [PATCH 02/10] Revert "cluster: don't collect region stats in API mode (#7817)" (#7874) ref tikv/pd#5839 Signed-off-by: Cabinfever_B --- server/cluster/cluster.go | 13 +++++++++---- tests/server/api/region_test.go | 27 --------------------------- 2 files changed, 9 insertions(+), 31 deletions(-) diff --git a/server/cluster/cluster.go b/server/cluster/cluster.go index ccc1d7fc7b5..c69e487c3db 100644 --- a/server/cluster/cluster.go +++ b/server/cluster/cluster.go @@ -1005,9 +1005,12 @@ func (c *RaftCluster) processRegionHeartbeat(region *core.RegionInfo) error { // Save to storage if meta is updated, except for flashback. // Save to cache if meta or leader is updated, or contains any down/pending peer. saveKV, saveCache, needSync := regionGuide(region, origin) - if !c.IsServiceIndependent(mcsutils.SchedulingServiceName) && !saveKV && !saveCache { + if !saveKV && !saveCache { // Due to some config changes need to update the region stats as well, // so we do some extra checks here. + // TODO: Due to the accuracy requirements of the API "/regions/check/xxx", + // region stats needs to be collected in API mode. + // We need to think of a better way to reduce this part of the cost in the future. if hasRegionStats && c.regionStats.RegionStatsNeedUpdate(region) { c.regionStats.Observe(region, c.getRegionStoresLocked(region)) } @@ -1035,9 +1038,11 @@ func (c *RaftCluster) processRegionHeartbeat(region *core.RegionInfo) error { } regionUpdateCacheEventCounter.Inc() } - if !c.IsServiceIndependent(mcsutils.SchedulingServiceName) { - cluster.Collect(c, region, c.GetRegionStores(region), hasRegionStats) - } + + // TODO: Due to the accuracy requirements of the API "/regions/check/xxx", + // region stats needs to be collected in API mode. + // We need to think of a better way to reduce this part of the cost in the future. + cluster.Collect(c, region, c.GetRegionStores(region), hasRegionStats) if c.storage != nil { // If there are concurrent heartbeats from the same region, the last write will win even if diff --git a/tests/server/api/region_test.go b/tests/server/api/region_test.go index ccac726027c..8c286dc12e2 100644 --- a/tests/server/api/region_test.go +++ b/tests/server/api/region_test.go @@ -28,7 +28,6 @@ import ( "github.com/stretchr/testify/suite" "github.com/tikv/pd/pkg/core" "github.com/tikv/pd/pkg/schedule/placement" - "github.com/tikv/pd/pkg/statistics" tu "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/tests" ) @@ -173,32 +172,6 @@ func (suite *regionTestSuite) checkAccelerateRegionsScheduleInRange(cluster *tes re.Len(idList, 2, len(idList)) } -func (suite *regionTestSuite) TestRegionStats() { - env := tests.NewSchedulingTestEnvironment(suite.T()) - env.RunTestInAPIMode(suite.checkRegionStats) - env.Cleanup() -} - -func (suite *regionTestSuite) checkRegionStats(cluster *tests.TestCluster) { - re := suite.Require() - leader := cluster.GetLeaderServer() - rc := leader.GetRaftCluster() - re.NotNil(rc) - for i := 13; i <= 16; i++ { - s1 := &metapb.Store{ - Id: uint64(i), - State: metapb.StoreState_Up, - NodeState: metapb.NodeState_Serving, - } - tests.MustPutStore(re, cluster, s1) - } - r := core.NewTestRegionInfo(1001, 13, []byte("b1"), []byte("b2"), core.SetApproximateSize(0)) - r.GetMeta().Peers = append(r.GetMeta().Peers, &metapb.Peer{Id: 5, StoreId: 14}, &metapb.Peer{Id: 6, StoreId: 15}) - tests.MustPutRegionInfo(re, cluster, r) - suite.checkRegionCount(re, cluster, 1) - re.False(rc.GetRegionStats().IsRegionStatsType(1001, statistics.EmptyRegion)) -} - func (suite *regionTestSuite) TestAccelerateRegionsScheduleInRanges() { suite.env.RunTestInTwoModes(suite.checkAccelerateRegionsScheduleInRanges) } From ba33aa5fb5bfbf59d3bf2bd0e0a46daae63e48b7 Mon Sep 17 00:00:00 2001 From: Yongbo Jiang Date: Tue, 5 Mar 2024 15:39:35 +0800 Subject: [PATCH 03/10] client: use Tracer from Span and fix unfinished `span` (#7860) close tikv/pd#7861 Signed-off-by: Cabinfever_B Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 64 ++++++++++++------------ client/gc_client.go | 8 +-- client/keyspace_client.go | 12 ++--- client/meta_storage_client.go | 8 +-- client/tso_dispatcher.go | 35 ++++--------- tests/integrations/client/client_test.go | 18 +++++++ tests/integrations/go.mod | 3 +- tests/integrations/go.sum | 8 +++ 8 files changed, 84 insertions(+), 72 deletions(-) diff --git a/client/client.go b/client/client.go index 5231d9f3497..81bf809ef4d 100644 --- a/client/client.go +++ b/client/client.go @@ -773,10 +773,10 @@ func (c *client) GetTSAsync(ctx context.Context) TSFuture { } func (c *client) GetLocalTSAsync(ctx context.Context, dcLocation string) TSFuture { - defer trace.StartRegion(ctx, "GetLocalTSAsync").End() - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("GetLocalTSAsync", opentracing.ChildOf(span.Context())) - ctx = opentracing.ContextWithSpan(ctx, span) + defer trace.StartRegion(ctx, "pdclient.GetLocalTSAsync").End() + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetLocalTSAsync", opentracing.ChildOf(span.Context())) + defer span.Finish() } req := tsoReqPool.Get().(*tsoRequest) @@ -875,8 +875,8 @@ func handleRegionResponse(res *pdpb.GetRegionResponse) *Region { } 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())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetRegionFromMember", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -913,8 +913,8 @@ func (c *client) GetRegionFromMember(ctx context.Context, key []byte, memberURLs } func (c *client) GetRegion(ctx context.Context, key []byte, opts ...GetRegionOption) (*Region, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetRegion", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetRegion", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -951,8 +951,8 @@ func (c *client) GetRegion(ctx context.Context, key []byte, opts ...GetRegionOpt } func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...GetRegionOption) (*Region, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetPrevRegion", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetPrevRegion", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -989,8 +989,8 @@ func (c *client) GetPrevRegion(ctx context.Context, key []byte, opts ...GetRegio } func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...GetRegionOption) (*Region, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetRegionByID", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetRegionByID", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1027,8 +1027,8 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64, opts ...Get } 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())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.ScanRegions", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1102,8 +1102,8 @@ func handleRegionsResponse(resp *pdpb.ScanRegionsResponse) []*Region { } func (c *client) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetStore", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetStore", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1146,8 +1146,8 @@ func (c *client) GetAllStores(ctx context.Context, opts ...GetStoreOption) ([]*m opt(options) } - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetAllStores", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetAllStores", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1173,8 +1173,8 @@ func (c *client) GetAllStores(ctx context.Context, opts ...GetStoreOption) ([]*m } func (c *client) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.UpdateGCSafePoint", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.UpdateGCSafePoint", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1204,8 +1204,8 @@ func (c *client) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint6 // determine the safepoint for multiple services, it does not trigger a GC // job. Use UpdateGCSafePoint to trigger the GC job if needed. func (c *client) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.UpdateServiceGCSafePoint", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.UpdateServiceGCSafePoint", opentracing.ChildOf(span.Context())) defer span.Finish() } @@ -1234,8 +1234,8 @@ func (c *client) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, } func (c *client) ScatterRegion(ctx context.Context, regionID uint64) error { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.ScatterRegion", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.ScatterRegion", opentracing.ChildOf(span.Context())) defer span.Finish() } return c.scatterRegionsWithGroup(ctx, regionID, "") @@ -1268,16 +1268,16 @@ func (c *client) scatterRegionsWithGroup(ctx context.Context, regionID uint64, g } func (c *client) ScatterRegions(ctx context.Context, regionsID []uint64, opts ...RegionsOption) (*pdpb.ScatterRegionResponse, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.ScatterRegions", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.ScatterRegions", opentracing.ChildOf(span.Context())) defer span.Finish() } return c.scatterRegionsWithOptions(ctx, regionsID, opts...) } func (c *client) SplitAndScatterRegions(ctx context.Context, splitKeys [][]byte, opts ...RegionsOption) (*pdpb.SplitAndScatterRegionsResponse, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.SplitAndScatterRegions", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.SplitAndScatterRegions", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1304,8 +1304,8 @@ func (c *client) SplitAndScatterRegions(ctx context.Context, splitKeys [][]byte, } func (c *client) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.GetOperator", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.GetOperator", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -1327,8 +1327,8 @@ func (c *client) GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOpe // SplitRegions split regions by given split keys func (c *client) SplitRegions(ctx context.Context, splitKeys [][]byte, opts ...RegionsOption) (*pdpb.SplitRegionsResponse, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.SplitRegions", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.SplitRegions", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() diff --git a/client/gc_client.go b/client/gc_client.go index fff292405c2..21eb0051499 100644 --- a/client/gc_client.go +++ b/client/gc_client.go @@ -34,8 +34,8 @@ type GCClient interface { // UpdateGCSafePointV2 update gc safe point for the given keyspace. func (c *client) UpdateGCSafePointV2(ctx context.Context, keyspaceID uint32, safePoint uint64) (uint64, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.UpdateGCSafePointV2", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.UpdateGCSafePointV2", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -63,8 +63,8 @@ func (c *client) UpdateGCSafePointV2(ctx context.Context, keyspaceID uint32, saf // UpdateServiceSafePointV2 update service safe point for the given keyspace. func (c *client) UpdateServiceSafePointV2(ctx context.Context, keyspaceID uint32, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.UpdateServiceSafePointV2", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.UpdateServiceSafePointV2", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() diff --git a/client/keyspace_client.go b/client/keyspace_client.go index fedb7452412..80b9cb0e436 100644 --- a/client/keyspace_client.go +++ b/client/keyspace_client.go @@ -45,8 +45,8 @@ func (c *client) keyspaceClient() keyspacepb.KeyspaceClient { // LoadKeyspace loads and returns target keyspace's metadata. func (c *client) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.KeyspaceMeta, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("keyspaceClient.LoadKeyspace", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("keyspaceClient.LoadKeyspace", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -84,8 +84,8 @@ func (c *client) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.Key // // Updated keyspace meta will be returned. func (c *client) UpdateKeyspaceState(ctx context.Context, id uint32, state keyspacepb.KeyspaceState) (*keyspacepb.KeyspaceMeta, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("keyspaceClient.UpdateKeyspaceState", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("keyspaceClient.UpdateKeyspaceState", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -123,8 +123,8 @@ func (c *client) WatchKeyspaces(ctx context.Context) (chan []*keyspacepb.Keyspac // GetAllKeyspaces get all keyspaces metadata. func (c *client) GetAllKeyspaces(ctx context.Context, startID uint32, limit uint32) ([]*keyspacepb.KeyspaceMeta, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("keyspaceClient.GetAllKeyspaces", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("keyspaceClient.GetAllKeyspaces", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() diff --git a/client/meta_storage_client.go b/client/meta_storage_client.go index b203fb914d3..8b158af2212 100644 --- a/client/meta_storage_client.go +++ b/client/meta_storage_client.go @@ -110,8 +110,8 @@ func (c *client) Put(ctx context.Context, key, value []byte, opts ...OpOption) ( opt(options) } - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.Put", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.Put", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() @@ -148,8 +148,8 @@ func (c *client) Get(ctx context.Context, key []byte, opts ...OpOption) (*meta_s options.rangeEnd = getPrefix(key) } - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("pdclient.Get", opentracing.ChildOf(span.Context())) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.Get", opentracing.ChildOf(span.Context())) defer span.Finish() } start := time.Now() diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 3930c9b1f33..3159a77d135 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -82,7 +82,7 @@ func (c *tsoClient) dispatchRequest(ctx context.Context, dcLocation string, requ return err } - defer trace.StartRegion(request.requestCtx, "tsoReqEnqueue").End() + defer trace.StartRegion(request.requestCtx, "pdclient.tsoReqEnqueue").End() select { case <-ctx.Done(): return ctx.Err() @@ -104,7 +104,7 @@ func (req *tsoRequest) Wait() (physical int64, logical int64, err error) { cmdDurationTSOAsyncWait.Observe(start.Sub(req.start).Seconds()) select { case err = <-req.done: - defer trace.StartRegion(req.requestCtx, "tsoReqDone").End() + defer trace.StartRegion(req.requestCtx, "pdclient.tsoReqDone").End() err = errors.WithStack(err) defer tsoReqPool.Put(req) if err != nil { @@ -350,7 +350,6 @@ func (c *tsoClient) handleDispatcher( cancel context.CancelFunc // addr -> connectionContext connectionCtxs sync.Map - opts []opentracing.StartSpanOption ) defer func() { log.Info("[tso] exit tso dispatcher", zap.String("dc-location", dc)) @@ -499,8 +498,7 @@ tsoBatchLoop: return case tsDeadlineCh.(chan *deadline) <- dl: } - opts = extractSpanReference(tbc, opts[:0]) - err = c.processRequests(stream, dc, tbc, opts) + err = c.processRequests(stream, dc, tbc) close(done) // If error happens during tso stream handling, reset stream and run the next trial. if err != nil { @@ -758,26 +756,16 @@ func (c *tsoClient) tryConnectToTSOWithProxy(dispatcherCtx context.Context, dc s return nil } -func extractSpanReference(tbc *tsoBatchController, opts []opentracing.StartSpanOption) []opentracing.StartSpanOption { - for _, req := range tbc.getCollectedRequests() { - if span := opentracing.SpanFromContext(req.requestCtx); span != nil { - opts = append(opts, opentracing.ChildOf(span.Context())) - } - } - return opts -} - func (c *tsoClient) processRequests( - stream tsoStream, dcLocation string, tbc *tsoBatchController, opts []opentracing.StartSpanOption, + stream tsoStream, dcLocation string, tbc *tsoBatchController, ) error { - if len(opts) > 0 { - span := opentracing.StartSpan("pdclient.processRequests", opts...) - defer span.Finish() - } - requests := tbc.getCollectedRequests() for _, req := range requests { - defer trace.StartRegion(req.requestCtx, "tsoReqSend").End() + defer trace.StartRegion(req.requestCtx, "pdclient.tsoReqSend").End() + if span := opentracing.SpanFromContext(req.requestCtx); span != nil && span.Tracer() != nil { + span = span.Tracer().StartSpan("pdclient.processRequests", opentracing.ChildOf(span.Context())) + defer span.Finish() + } } count := int64(len(requests)) reqKeyspaceGroupID := c.svcDiscovery.GetKeyspaceGroupID() @@ -849,11 +837,8 @@ func (c *tsoClient) compareAndSwapTS( func (c *tsoClient) finishRequest(requests []*tsoRequest, physical, firstLogical int64, suffixBits uint32, err error) { for i := 0; i < len(requests); i++ { - if span := opentracing.SpanFromContext(requests[i].requestCtx); span != nil { - span.Finish() - } requests[i].physical, requests[i].logical = physical, tsoutil.AddLogical(firstLogical, int64(i), suffixBits) - defer trace.StartRegion(requests[i].requestCtx, "tsoReqDequeue").End() + defer trace.StartRegion(requests[i].requestCtx, "pdclient.tsoReqDequeue").End() requests[i].done <- err } } diff --git a/tests/integrations/client/client_test.go b/tests/integrations/client/client_test.go index b66b15d8243..ca6776aec71 100644 --- a/tests/integrations/client/client_test.go +++ b/tests/integrations/client/client_test.go @@ -30,6 +30,8 @@ import ( "time" "github.com/docker/go-units" + "github.com/opentracing/basictracer-go" + "github.com/opentracing/opentracing-go" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/kvproto/pkg/metapb" @@ -469,6 +471,22 @@ func TestGlobalAndLocalTSO(t *testing.T) { re.NoError(err) re.NoError(failpoint.Disable("github.com/tikv/pd/client/skipUpdateMember")) + recorder := basictracer.NewInMemoryRecorder() + tracer := basictracer.New(recorder) + span := tracer.StartSpan("trace") + ctx = opentracing.ContextWithSpan(ctx, span) + future := cli.GetLocalTSAsync(ctx, "error-dc") + spans := recorder.GetSpans() + re.Len(spans, 1) + _, _, err = future.Wait() + re.Error(err) + spans = recorder.GetSpans() + re.Len(spans, 1) + _, _, err = cli.GetTS(ctx) + re.NoError(err) + spans = recorder.GetSpans() + re.Len(spans, 3) + // Test the TSO follower proxy while enabling the Local TSO. cli.UpdateOption(pd.EnableTSOFollowerProxy, true) // Sleep a while here to prevent from canceling the ongoing TSO request. diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index f1da9295e8f..2f6e7e71eb5 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -122,7 +122,8 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect 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/opentracing/basictracer-go v1.1.0 + github.com/opentracing/opentracing-go v1.2.0 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 diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index dbcffd14b42..a1c9fcccf91 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -193,6 +193,7 @@ github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5 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= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt v3.2.1+incompatible h1:73Z+4BJcrTC+KczS6WvTPvRGOp1WmfEP4Q1lOd9Z/+c= @@ -310,6 +311,7 @@ github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHm github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= 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= @@ -382,6 +384,9 @@ github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042 github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= 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/basictracer-go v1.1.0 h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0= +github.com/opentracing/basictracer-go v1.1.0/go.mod h1:V2HZueSJEp879yv285Aap1BS69fQMD+MNP1mRs6mBQc= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= 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/otiai10/copy v1.7.0/go.mod h1:rmRl6QPdJj6EiUqXQ/4Nn2lLXoNQjFCQbbNrxgc/t3U= @@ -634,6 +639,7 @@ golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTk golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5 h1:2M3HP5CCK1Si9FQhwnzYhXdG6DXeebvUHFpre8QvbyI= golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= @@ -659,6 +665,7 @@ golang.org/x/net v0.0.0-20190522155817-f3200d17e092/go.mod h1:HSz+uSET+XFnRR8LxR golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= 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-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= @@ -739,6 +746,7 @@ golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= From 6a4573481cccbf3dea455e6343bdd535608456e0 Mon Sep 17 00:00:00 2001 From: Yongbo Jiang Date: Tue, 5 Mar 2024 15:51:35 +0800 Subject: [PATCH 04/10] test: fix unstable test TestLevelDBBackend (#7876) close tikv/pd#7875 Signed-off-by: Cabinfever_B Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/storage/leveldb_backend_test.go | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pkg/storage/leveldb_backend_test.go b/pkg/storage/leveldb_backend_test.go index f727dd69ee3..45af7201c85 100644 --- a/pkg/storage/leveldb_backend_test.go +++ b/pkg/storage/leveldb_backend_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/utils/testutil" ) func TestLevelDBBackend(t *testing.T) { @@ -86,10 +87,11 @@ func TestLevelDBBackend(t *testing.T) { val, err = backend.Load(key) re.NoError(err) re.Empty(val) - time.Sleep(defaultDirtyFlushTick * 2) - val, err = backend.Load(key) - re.NoError(err) - re.Equal(value, val) + testutil.Eventually(re, func() bool { + val, err = backend.Load(key) + re.NoError(err) + return value == val + }, testutil.WithWaitFor(defaultDirtyFlushTick*5), testutil.WithTickInterval(defaultDirtyFlushTick/2)) err = backend.Remove(key) re.NoError(err) val, err = backend.Load(key) From e72d49b97cf52d2b91f30d40361e55f34d020628 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 6 Mar 2024 13:57:34 +0800 Subject: [PATCH 05/10] metrics: update tso metrics in micro service (#7873) close tikv/pd#5839 Signed-off-by: lhy1024 Co-authored-by: Hu# --- metrics/grafana/pd.json | 67 +++++++++++++++++++++++++++++++++-------- 1 file changed, 55 insertions(+), 12 deletions(-) diff --git a/metrics/grafana/pd.json b/metrics/grafana/pd.json index e70c1eb89ef..2ead4b1e249 100644 --- a/metrics/grafana/pd.json +++ b/metrics/grafana/pd.json @@ -1809,7 +1809,7 @@ "steppedLine": false, "targets": [ { - "expr": "irate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}[30s])", + "expr": "irate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}[30s])", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1819,7 +1819,7 @@ }, { "exemplar": true, - "expr": "pd_service_maxprocs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "pd_service_maxprocs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": false, "interval": "", "legendFormat": "{{job}}-{{instance}}-limit", @@ -1917,7 +1917,7 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "format": "time_series", "hide": false, "interval": "", @@ -1927,42 +1927,42 @@ "step": 4 }, { - "expr": "go_memstats_heap_sys_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_heap_sys_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": true, "interval": "", "legendFormat": "HeapSys-{{job}}-{{instance}}", "refId": "B" }, { - "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": false, "interval": "", "legendFormat": "HeapInuse-{{job}}-{{instance}}", "refId": "C" }, { - "expr": "go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": true, "interval": "", "legendFormat": "HeapAlloc-{{job}}-{{instance}}", "refId": "D" }, { - "expr": "go_memstats_heap_idle_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_heap_idle_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": true, "interval": "", "legendFormat": "HeapIdle-{{job}}-{{instance}}", "refId": "E" }, { - "expr": "go_memstats_heap_released_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_heap_released_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": true, "interval": "", "legendFormat": "HeapReleased-{{job}}-{{instance}}", "refId": "F" }, { - "expr": "go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "hide": true, "interval": "", "legendFormat": "GCTrigger-{{job}}-{{instance}}", @@ -2059,7 +2059,7 @@ "steppedLine": false, "targets": [ { - "expr": "(time() - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"})", + "expr": "(time() - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"})", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -2158,7 +2158,7 @@ "steppedLine": false, "targets": [ { - "expr": "go_goroutines{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*pd.*\"}", + "expr": "go_goroutines{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\",job=~\".*(pd|tso|scheduling).*\"}", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -10618,6 +10618,40 @@ "intervalFactor": 1, "legendFormat": "99.999% tso", "refId": "D" + }, + { + "expr": "histogram_quantile(0.90, sum(rate(tso_server_handle_tso_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "90% tso", + "refId": "E", + "step": 2 + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tso_server_handle_tso_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "99% tso", + "refId": "F", + "step": 2 + }, + { + "expr": "histogram_quantile(0.999, sum(rate(tso_server_handle_tso_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "99.9% tso", + "refId": "G", + "step": 2 + }, + { + "expr": "histogram_quantile(0.99999, sum(rate(tso_server_handle_tso_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99.999% tso", + "refId": "H" } ], "thresholds": [], @@ -10830,6 +10864,15 @@ "hide": true, "refId": "B", "step": 2 + }, + { + "expr": "sum(rate(tso_server_handle_tso_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "tso request/secs", + "refId": "C", + "step": 2 } ], "thresholds": [], @@ -13590,4 +13633,4 @@ "title": "Test-Cluster-PD", "uid": "Q6RuHYIWk", "version": 1 -} +} \ No newline at end of file From 2574aaa0d0cfaba31df3c1528710962ca5cb6b13 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 7 Mar 2024 10:29:06 +0800 Subject: [PATCH 06/10] client/http: support update Keyspace safe point version (#7879) ref tikv/pd#7880 If safe_point_version is 'v2' is means the current KeySpace is enabled safe point v2. Signed-off-by: y_static_y@sina.com Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/http/api.go | 13 +++++ client/http/interface.go | 50 +++++++++++++++++++ client/http/request_info.go | 2 + client/http/types.go | 37 ++++++++++++++ tests/integrations/client/http_client_test.go | 27 ++++++++++ 5 files changed, 129 insertions(+) diff --git a/client/http/api.go b/client/http/api.go index 434bf0d1fe4..a1ca96b38f1 100644 --- a/client/http/api.go +++ b/client/http/api.go @@ -80,6 +80,9 @@ const ( operators = "/pd/api/v1/operators" // Micro Service microServicePrefix = "/pd/api/v2/ms" + // Keyspace + KeyspaceConfig = "/pd/api/v2/keyspaces/%s/config" + GetKeyspaceMetaByName = "/pd/api/v2/keyspaces/%s" ) // RegionByID returns the path of PD HTTP API to get region by ID. @@ -201,3 +204,13 @@ func MicroServiceMembers(service string) string { func MicroServicePrimary(service string) string { return fmt.Sprintf("%s/primary/%s", microServicePrefix, service) } + +// GetUpdateKeyspaceConfigURL returns the path of PD HTTP API to update keyspace config. +func GetUpdateKeyspaceConfigURL(keyspaceName string) string { + return fmt.Sprintf(KeyspaceConfig, keyspaceName) +} + +// GetKeyspaceMetaByNameURL returns the path of PD HTTP API to get keyspace meta by keyspace name. +func GetKeyspaceMetaByNameURL(keyspaceName string) string { + return fmt.Sprintf(GetKeyspaceMetaByName, keyspaceName) +} diff --git a/client/http/interface.go b/client/http/interface.go index 06beba85a45..f112cf8362a 100644 --- a/client/http/interface.go +++ b/client/http/interface.go @@ -23,6 +23,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/tikv/pd/client/retry" @@ -95,6 +96,10 @@ type Client interface { GetMicroServicePrimary(context.Context, string) (string, error) DeleteOperators(context.Context) error + /* Keyspace interface */ + UpdateKeyspaceSafePointVersion(ctx context.Context, keyspaceName string, keyspaceSafePointVersion *KeyspaceSafePointVersionConfig) error + GetKeyspaceMetaByName(ctx context.Context, keyspaceName string) (*keyspacepb.KeyspaceMeta, error) + /* Client-related methods */ // WithCallerID sets and returns a new client with the given caller ID. WithCallerID(string) Client @@ -900,3 +905,48 @@ func (c *client) DeleteOperators(ctx context.Context) error { WithURI(operators). WithMethod(http.MethodDelete)) } + +// UpdateKeyspaceSafePointVersion patches the keyspace config. +func (c *client) UpdateKeyspaceSafePointVersion(ctx context.Context, keyspaceName string, keyspaceSafePointVersion *KeyspaceSafePointVersionConfig) error { + keyspaceConfigPatchJSON, err := json.Marshal(keyspaceSafePointVersion) + if err != nil { + return errors.Trace(err) + } + return c.request(ctx, newRequestInfo(). + WithName(UpdateKeyspaceSafePointVersionName). + WithURI(GetUpdateKeyspaceConfigURL(keyspaceName)). + WithMethod(http.MethodPatch). + WithBody(keyspaceConfigPatchJSON)) +} + +// GetKeyspaceMetaByName get the given keyspace meta. +func (c *client) GetKeyspaceMetaByName(ctx context.Context, keyspaceName string) (*keyspacepb.KeyspaceMeta, error) { + var ( + tempKeyspaceMeta tempKeyspaceMeta + keyspaceMetaPB keyspacepb.KeyspaceMeta + ) + err := c.request(ctx, newRequestInfo(). + WithName(GetKeyspaceMetaByNameName). + WithURI(GetKeyspaceMetaByNameURL(keyspaceName)). + WithMethod(http.MethodGet). + WithResp(&tempKeyspaceMeta)) + + if err != nil { + return nil, err + } + + keyspaceState, err := stringToKeyspaceState(tempKeyspaceMeta.State) + if err != nil { + return nil, err + } + + keyspaceMetaPB = keyspacepb.KeyspaceMeta{ + Name: tempKeyspaceMeta.Name, + Id: tempKeyspaceMeta.ID, + Config: tempKeyspaceMeta.Config, + CreatedAt: tempKeyspaceMeta.CreatedAt, + StateChangedAt: tempKeyspaceMeta.StateChangedAt, + State: keyspaceState, + } + return &keyspaceMetaPB, nil +} diff --git a/client/http/request_info.go b/client/http/request_info.go index a7a6e9b7652..e5f1ee4c3f3 100644 --- a/client/http/request_info.go +++ b/client/http/request_info.go @@ -77,6 +77,8 @@ const ( setSnapshotRecoveringMarkName = "SetSnapshotRecoveringMark" deleteSnapshotRecoveringMarkName = "DeleteSnapshotRecoveringMark" deleteOperators = "DeleteOperators" + UpdateKeyspaceSafePointVersionName = "UpdateKeyspaceSafePointVersion" + GetKeyspaceMetaByNameName = "GetKeyspaceMetaByName" ) type requestInfo struct { diff --git a/client/http/types.go b/client/http/types.go index 56ad0427f12..7d2cbcfaaa8 100644 --- a/client/http/types.go +++ b/client/http/types.go @@ -17,10 +17,12 @@ package http import ( "encoding/hex" "encoding/json" + "fmt" "net/url" "time" "github.com/pingcap/kvproto/pkg/encryptionpb" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" ) @@ -601,3 +603,38 @@ type MicroServiceMember struct { DeployPath string `json:"deploy-path"` StartTimestamp int64 `json:"start-timestamp"` } + +// KeyspaceSafePointVersion represents parameters needed to modify the safe point version. +type KeyspaceSafePointVersion struct { + SafePointVersion string `json:"safe_point_version,omitempty"` +} + +// KeyspaceSafePointVersionConfig represents parameters needed to modify target keyspace's configs. +type KeyspaceSafePointVersionConfig struct { + Config KeyspaceSafePointVersion `json:"config"` +} + +// tempKeyspaceMeta is the keyspace meta struct that returned from the http interface. +type tempKeyspaceMeta struct { + ID uint32 `json:"id"` + Name string `json:"name"` + State string `json:"state"` + CreatedAt int64 `json:"created_at"` + StateChangedAt int64 `json:"state_changed_at"` + Config map[string]string `json:"config"` +} + +func stringToKeyspaceState(str string) (keyspacepb.KeyspaceState, error) { + switch str { + case "ENABLED": + return keyspacepb.KeyspaceState_ENABLED, nil + case "DISABLED": + return keyspacepb.KeyspaceState_DISABLED, nil + case "ARCHIVED": + return keyspacepb.KeyspaceState_ARCHIVED, nil + case "TOMBSTONE": + return keyspacepb.KeyspaceState_TOMBSTONE, nil + default: + return keyspacepb.KeyspaceState(0), fmt.Errorf("invalid KeyspaceState string: %s", str) + } +} diff --git a/tests/integrations/client/http_client_test.go b/tests/integrations/client/http_client_test.go index 9bcc7bc9fbd..64a29b39fe0 100644 --- a/tests/integrations/client/http_client_test.go +++ b/tests/integrations/client/http_client_test.go @@ -84,6 +84,7 @@ func (suite *httpClientTestSuite) SetupSuite() { leader := cluster.WaitLeader() re.NotEmpty(leader) leaderServer := cluster.GetLeaderServer() + err = leaderServer.BootstrapCluster() re.NoError(err) for _, region := range []*core.RegionInfo{ @@ -751,3 +752,29 @@ func (suite *httpClientTestSuite) TestRedirectWithMetrics() { re.Equal(float64(3), out.Counter.GetValue()) c.Close() } + +func (suite *httpClientTestSuite) TestUpdateKeyspaceSafePointVersion() { + suite.RunTestInTwoModes(suite.checkUpdateKeyspaceSafePointVersion) +} + +func (suite *httpClientTestSuite) checkUpdateKeyspaceSafePointVersion(mode mode, client pd.Client) { + re := suite.Require() + env := suite.env[mode] + + keyspaceName := "DEFAULT" + safePointVersion := "v2" + + keyspaceSafePointVersionConfig := pd.KeyspaceSafePointVersionConfig{ + Config: pd.KeyspaceSafePointVersion{ + SafePointVersion: safePointVersion, + }, + } + err := client.UpdateKeyspaceSafePointVersion(env.ctx, keyspaceName, &keyspaceSafePointVersionConfig) + re.NoError(err) + + keyspaceMetaRes, err := client.GetKeyspaceMetaByName(env.ctx, keyspaceName) + re.NoError(err) + val, ok := keyspaceMetaRes.Config["safe_point_version"] + re.True(ok) + re.Equal(safePointVersion, val) +} From 721b6fbf9f40152af811023ea836d87b1f35d1ea Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 7 Mar 2024 10:59:05 +0800 Subject: [PATCH 07/10] client/http, tests: utilize a mock PD service discovery during testing (#7883) ref tikv/pd#7300 Utilize a mock PD service discovery to prevent blocking of the HTTP request during testing. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/http/client.go | 15 ++++-- client/http/client_test.go | 18 ++++--- client/mock_pd_service_discovery.go | 74 +++++++++++++++++++++++++++++ client/pd_service_discovery.go | 6 ++- 4 files changed, 100 insertions(+), 13 deletions(-) create mode 100644 client/mock_pd_service_discovery.go diff --git a/client/http/client.go b/client/http/client.go index 389c5ee5bef..5ac00a8a43b 100644 --- a/client/http/client.go +++ b/client/http/client.go @@ -305,7 +305,8 @@ func NewClient( } sd := pd.NewDefaultPDServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) if err := sd.Init(); err != nil { - log.Error("[pd] init service discovery failed", zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) + log.Error("[pd] init service discovery failed", + zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) return nil } c.inner.init(sd) @@ -382,9 +383,8 @@ func NewHTTPClientWithRequestChecker(checker requestChecker) *http.Client { } } -// newClientWithoutInitServiceDiscovery creates a PD HTTP client -// with the given PD addresses and TLS config without init service discovery. -func newClientWithoutInitServiceDiscovery( +// newClientWithMockServiceDiscovery creates a new PD HTTP client with a mock PD service discovery. +func newClientWithMockServiceDiscovery( source string, pdAddrs []string, opts ...ClientOption, @@ -395,7 +395,12 @@ func newClientWithoutInitServiceDiscovery( for _, opt := range opts { opt(c) } - sd := pd.NewDefaultPDServiceDiscovery(ctx, cancel, pdAddrs, c.inner.tlsConf) + sd := pd.NewMockPDServiceDiscovery(pdAddrs, c.inner.tlsConf) + if err := sd.Init(); err != nil { + log.Error("[pd] init mock service discovery failed", + zap.String("source", source), zap.Strings("pd-addrs", pdAddrs), zap.Error(err)) + return nil + } c.inner.init(sd) return c } diff --git a/client/http/client_test.go b/client/http/client_test.go index 02fce93838e..49faefefaec 100644 --- a/client/http/client_test.go +++ b/client/http/client_test.go @@ -28,6 +28,7 @@ import ( func TestPDAllowFollowerHandleHeader(t *testing.T) { re := require.New(t) + checked := 0 httpClient := NewHTTPClientWithRequestChecker(func(req *http.Request) error { var expectedVal string if req.URL.Path == HotHistory { @@ -38,16 +39,19 @@ func TestPDAllowFollowerHandleHeader(t *testing.T) { re.Failf("PD allow follower handler header check failed", "should be %s, but got %s", expectedVal, val) } + checked++ return nil }) - c := newClientWithoutInitServiceDiscovery("test-header", []string{"http://127.0.0.1"}, WithHTTPClient(httpClient)) + c := newClientWithMockServiceDiscovery("test-header", []string{"http://127.0.0.1"}, WithHTTPClient(httpClient)) + defer c.Close() c.GetRegions(context.Background()) c.GetHistoryHotRegions(context.Background(), &HistoryHotRegionsRequest{}) - c.Close() + re.Equal(2, checked) } func TestCallerID(t *testing.T) { re := require.New(t) + checked := 0 expectedVal := atomic.NewString(defaultCallerID) httpClient := NewHTTPClientWithRequestChecker(func(req *http.Request) error { val := req.Header.Get(xCallerIDKey) @@ -56,20 +60,23 @@ func TestCallerID(t *testing.T) { re.Failf("Caller ID header check failed", "should be %s, but got %s", expectedVal, val) } + checked++ return nil }) - c := newClientWithoutInitServiceDiscovery("test-caller-id", []string{"http://127.0.0.1"}, WithHTTPClient(httpClient)) + c := newClientWithMockServiceDiscovery("test-caller-id", []string{"http://127.0.0.1"}, WithHTTPClient(httpClient)) + defer c.Close() c.GetRegions(context.Background()) expectedVal.Store("test") c.WithCallerID(expectedVal.Load()).GetRegions(context.Background()) - c.Close() + re.Equal(2, checked) } func TestWithBackoffer(t *testing.T) { re := require.New(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - c := newClientWithoutInitServiceDiscovery("test-with-backoffer", []string{"http://127.0.0.1"}) + c := newClientWithMockServiceDiscovery("test-with-backoffer", []string{"http://127.0.0.1"}) + defer c.Close() base := 100 * time.Millisecond max := 500 * time.Millisecond @@ -88,5 +95,4 @@ func TestWithBackoffer(t *testing.T) { _, err = c.WithBackoffer(bo).GetPDVersion(timeoutCtx) re.InDelta(3*time.Second, time.Since(start), float64(250*time.Millisecond)) re.ErrorIs(err, context.DeadlineExceeded) - c.Close() } diff --git a/client/mock_pd_service_discovery.go b/client/mock_pd_service_discovery.go new file mode 100644 index 00000000000..10f7f080106 --- /dev/null +++ b/client/mock_pd_service_discovery.go @@ -0,0 +1,74 @@ +// Copyright 2024 TiKV Project Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pd + +import ( + "crypto/tls" + "sync" + + "google.golang.org/grpc" +) + +var _ ServiceDiscovery = (*mockPDServiceDiscovery)(nil) + +type mockPDServiceDiscovery struct { + urls []string + tlsCfg *tls.Config + clients []ServiceClient +} + +// NewMockPDServiceDiscovery creates a mock PD service discovery. +func NewMockPDServiceDiscovery(urls []string, tlsCfg *tls.Config) *mockPDServiceDiscovery { + return &mockPDServiceDiscovery{ + urls: urls, + tlsCfg: tlsCfg, + } +} + +// Init directly creates the service clients with the given URLs. +func (m *mockPDServiceDiscovery) Init() error { + m.clients = make([]ServiceClient, 0, len(m.urls)) + for _, url := range m.urls { + m.clients = append(m.clients, newPDServiceClient(url, url, m.tlsCfg, nil, false)) + } + return nil +} + +// Close clears the service clients. +func (m *mockPDServiceDiscovery) Close() { + clear(m.clients) +} + +// GetAllServiceClients returns all service clients init in the mock PD service discovery. +func (m *mockPDServiceDiscovery) GetAllServiceClients() []ServiceClient { + return m.clients +} + +func (m *mockPDServiceDiscovery) GetClusterID() uint64 { return 0 } +func (m *mockPDServiceDiscovery) GetKeyspaceID() uint32 { return 0 } +func (m *mockPDServiceDiscovery) GetKeyspaceGroupID() uint32 { return 0 } +func (m *mockPDServiceDiscovery) GetServiceURLs() []string { return nil } +func (m *mockPDServiceDiscovery) GetServingEndpointClientConn() *grpc.ClientConn { return nil } +func (m *mockPDServiceDiscovery) GetClientConns() *sync.Map { return nil } +func (m *mockPDServiceDiscovery) GetServingAddr() string { return "" } +func (m *mockPDServiceDiscovery) GetBackupAddrs() []string { return nil } +func (m *mockPDServiceDiscovery) GetServiceClient() ServiceClient { return nil } +func (m *mockPDServiceDiscovery) GetOrCreateGRPCConn(addr string) (*grpc.ClientConn, error) { + return nil, nil +} +func (m *mockPDServiceDiscovery) ScheduleCheckMemberChanged() {} +func (m *mockPDServiceDiscovery) CheckMemberChanged() error { return nil } +func (m *mockPDServiceDiscovery) AddServingAddrSwitchedCallback(callbacks ...func()) {} +func (m *mockPDServiceDiscovery) AddServiceAddrsSwitchedCallback(callbacks ...func()) {} diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index 137a5c640c0..5d9105e7681 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -424,8 +424,10 @@ type tsoAllocatorEventSource interface { SetTSOGlobalServAddrUpdatedCallback(callback tsoGlobalServAddrUpdatedFunc) } -var _ ServiceDiscovery = (*pdServiceDiscovery)(nil) -var _ tsoAllocatorEventSource = (*pdServiceDiscovery)(nil) +var ( + _ ServiceDiscovery = (*pdServiceDiscovery)(nil) + _ tsoAllocatorEventSource = (*pdServiceDiscovery)(nil) +) // pdServiceDiscovery is the service discovery client of PD/API service which is quorum based type pdServiceDiscovery struct { From 32bba44b9162a1909f438329b3773e8d745975dd Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 7 Mar 2024 11:09:35 +0800 Subject: [PATCH 08/10] *: remove uuid dependency (#7885) ref tikv/pd#5836 Signed-off-by: Ryan Leung Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- go.mod | 2 +- pkg/mcs/tso/server/server.go | 2 +- pkg/tso/keyspace_group_manager.go | 4 +- pkg/tso/keyspace_group_manager_test.go | 53 ++++++++++++++------------ 4 files changed, 32 insertions(+), 29 deletions(-) diff --git a/go.mod b/go.mod index 6e7c411802f..15934faff3a 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,6 @@ require ( github.com/gin-gonic/gin v1.9.1 github.com/gogo/protobuf v1.3.2 github.com/google/btree v1.1.2 - github.com/google/uuid v1.3.0 github.com/gorilla/mux v1.7.4 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/joho/godotenv v1.4.0 @@ -115,6 +114,7 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/pprof v0.0.0-20211122183932-1daafda22083 // indirect github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect + github.com/google/uuid v1.3.0 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 4df43e1ebfc..bac93ddfb6b 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -381,7 +381,7 @@ func (s *Server) startServer() (err error) { } s.keyspaceGroupManager = tso.NewKeyspaceGroupManager( s.serverLoopCtx, s.serviceID, s.GetClient(), s.GetHTTPClient(), s.cfg.AdvertiseListenAddr, - discovery.TSOPath(s.clusterID), legacySvcRootPath, tsoSvcRootPath, s.cfg) + s.clusterID, legacySvcRootPath, tsoSvcRootPath, s.cfg) if err := s.keyspaceGroupManager.Initialize(); err != nil { return err } diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index b752233fcff..d259ab27a5b 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -398,7 +398,7 @@ func NewKeyspaceGroupManager( etcdClient *clientv3.Client, httpClient *http.Client, electionNamePrefix string, - tsoServiceKey string, + clusterID uint64, legacySvcRootPath string, tsoSvcRootPath string, cfg ServiceConfig, @@ -417,7 +417,7 @@ func NewKeyspaceGroupManager( etcdClient: etcdClient, httpClient: httpClient, electionNamePrefix: electionNamePrefix, - tsoServiceKey: tsoServiceKey, + tsoServiceKey: discovery.TSOPath(clusterID), legacySvcRootPath: legacySvcRootPath, tsoSvcRootPath: tsoSvcRootPath, primaryPriorityCheckInterval: defaultPrimaryPriorityCheckInterval, diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index d3d5f8256e6..7af76ee6c28 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -22,12 +22,12 @@ import ( "path" "reflect" "sort" + "strconv" "strings" "sync" "testing" "time" - "github.com/google/uuid" "github.com/pingcap/failpoint" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -152,15 +152,16 @@ func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { re := suite.Require() tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: suite.cfg.AdvertiseListenAddr} - guid := uuid.New().String() - tsoServiceKey := discovery.ServicePath(guid, "tso") - legacySvcRootPath := path.Join("/pd", guid) - tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, guid, "tso") - electionNamePrefix := "tso-server-" + guid + clusterID := rand.Uint64() + clusterIDStr := strconv.FormatUint(clusterID, 10) + + legacySvcRootPath := path.Join("/pd", clusterIDStr) + tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, clusterIDStr, "tso") + electionNamePrefix := "tso-server-" + clusterIDStr kgm := NewKeyspaceGroupManager( suite.ctx, tsoServiceID, suite.etcdClient, nil, electionNamePrefix, - tsoServiceKey, legacySvcRootPath, tsoSvcRootPath, suite.cfg) + clusterID, legacySvcRootPath, tsoSvcRootPath, suite.cfg) defer kgm.Close() err := kgm.Initialize() re.NoError(err) @@ -757,7 +758,7 @@ func (suite *keyspaceGroupManagerTestSuite) runTestLoadKeyspaceGroupsAssignment( if assignToMe { svcAddrs = append(svcAddrs, mgr.tsoServiceID.ServiceAddr) } else { - svcAddrs = append(svcAddrs, uuid.NewString()) + svcAddrs = append(svcAddrs, fmt.Sprintf("test-%d", rand.Uint64())) } addKeyspaceGroupAssignment( suite.ctx, suite.etcdClient, uint32(j), mgr.legacySvcRootPath, @@ -787,23 +788,23 @@ func (suite *keyspaceGroupManagerTestSuite) runTestLoadKeyspaceGroupsAssignment( func (suite *keyspaceGroupManagerTestSuite) newUniqueKeyspaceGroupManager( loadKeyspaceGroupsBatchSize int64, // set to 0 to use the default value ) *KeyspaceGroupManager { - return suite.newKeyspaceGroupManager(loadKeyspaceGroupsBatchSize, uuid.New().String(), suite.cfg) + return suite.newKeyspaceGroupManager(loadKeyspaceGroupsBatchSize, rand.Uint64(), suite.cfg) } func (suite *keyspaceGroupManagerTestSuite) newKeyspaceGroupManager( loadKeyspaceGroupsBatchSize int64, // set to 0 to use the default value - uniqueStr string, + clusterID uint64, cfg *TestServiceConfig, ) *KeyspaceGroupManager { tsoServiceID := &discovery.ServiceRegistryEntry{ServiceAddr: cfg.GetAdvertiseListenAddr()} - tsoServiceKey := discovery.ServicePath(uniqueStr, "tso") - legacySvcRootPath := path.Join("/pd", uniqueStr) - tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, uniqueStr, "tso") + clusterIDStr := strconv.FormatUint(clusterID, 10) + legacySvcRootPath := path.Join("/pd", clusterIDStr) + tsoSvcRootPath := path.Join(mcsutils.MicroserviceRootPath, clusterIDStr, "tso") electionNamePrefix := "kgm-test-" + cfg.GetAdvertiseListenAddr() kgm := NewKeyspaceGroupManager( suite.ctx, tsoServiceID, suite.etcdClient, nil, electionNamePrefix, - tsoServiceKey, legacySvcRootPath, tsoSvcRootPath, cfg) + clusterID, legacySvcRootPath, tsoSvcRootPath, cfg) if loadKeyspaceGroupsBatchSize != 0 { kgm.loadKeyspaceGroupsBatchSize = loadKeyspaceGroupsBatchSize } @@ -1043,18 +1044,20 @@ func (suite *keyspaceGroupManagerTestSuite) TestPrimaryPriorityChange() { var err error defaultPriority := mcsutils.DefaultKeyspaceGroupReplicaPriority - uniqueStr := uuid.New().String() - rootPath := path.Join("/pd", uniqueStr) + clusterID := rand.Uint64() + clusterIDStr := strconv.FormatUint(clusterID, 10) + + rootPath := path.Join("/pd", clusterIDStr) cfg1 := suite.createConfig() cfg2 := suite.createConfig() svcAddr1 := cfg1.GetAdvertiseListenAddr() svcAddr2 := cfg2.GetAdvertiseListenAddr() // Register TSO server 1 - err = suite.registerTSOServer(re, uniqueStr, svcAddr1, cfg1) + err = suite.registerTSOServer(re, clusterIDStr, svcAddr1, cfg1) re.NoError(err) defer func() { - re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr1)) + re.NoError(suite.deregisterTSOServer(clusterIDStr, svcAddr1)) }() // Create three keyspace groups on two TSO servers with default replica priority. @@ -1067,7 +1070,7 @@ func (suite *keyspaceGroupManagerTestSuite) TestPrimaryPriorityChange() { // Create the first TSO server which loads all three keyspace groups created above. // All primaries should be on the first TSO server. - mgr1 := suite.newKeyspaceGroupManager(1, uniqueStr, cfg1) + mgr1 := suite.newKeyspaceGroupManager(1, clusterID, cfg1) re.NotNil(mgr1) defer mgr1.Close() err = mgr1.Initialize() @@ -1099,9 +1102,9 @@ func (suite *keyspaceGroupManagerTestSuite) TestPrimaryPriorityChange() { checkTSO(ctx, re, &wg, mgr1, ids) // Create the Second TSO server. - err = suite.registerTSOServer(re, uniqueStr, svcAddr2, cfg2) + err = suite.registerTSOServer(re, clusterIDStr, svcAddr2, cfg2) re.NoError(err) - mgr2 := suite.newKeyspaceGroupManager(1, uniqueStr, cfg2) + mgr2 := suite.newKeyspaceGroupManager(1, clusterID, cfg2) re.NotNil(mgr2) err = mgr2.Initialize() re.NoError(err) @@ -1110,17 +1113,17 @@ func (suite *keyspaceGroupManagerTestSuite) TestPrimaryPriorityChange() { // Shutdown the second TSO server. mgr2.Close() - re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr2)) + re.NoError(suite.deregisterTSOServer(clusterIDStr, svcAddr2)) // The primaries should move back to the first TSO server. waitForPrimariesServing(re, []*KeyspaceGroupManager{mgr1, mgr1, mgr1}, ids) // Restart the Second TSO server. - err = suite.registerTSOServer(re, uniqueStr, svcAddr2, cfg2) + err = suite.registerTSOServer(re, clusterIDStr, svcAddr2, cfg2) re.NoError(err) defer func() { - re.NoError(suite.deregisterTSOServer(uniqueStr, svcAddr2)) + re.NoError(suite.deregisterTSOServer(clusterIDStr, svcAddr2)) }() - mgr2 = suite.newKeyspaceGroupManager(1, uniqueStr, cfg2) + mgr2 = suite.newKeyspaceGroupManager(1, clusterID, cfg2) re.NotNil(mgr2) defer mgr2.Close() err = mgr2.Initialize() From 57cd603481e54b2eb83ec36c046c26b1a808393e Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 7 Mar 2024 11:47:05 +0800 Subject: [PATCH 09/10] *: upgrade etcd to v3.4.30 (#7884) ref tikv/pd#5520 Upgrade etcd to v3.4.30 and adopt the latest code changes. Signed-off-by: JmPotato --- client/grpcutil/grpcutil.go | 3 +- client/pd_service_discovery_test.go | 5 +- go.mod | 20 ++++--- go.sum | 50 ++++-------------- pkg/dashboard/adapter/config.go | 2 +- pkg/election/leadership_test.go | 6 +-- .../scheduling/server/rule/watcher_test.go | 2 +- pkg/mcs/tso/server/testutil.go | 3 +- pkg/syncer/client_test.go | 2 +- pkg/tso/keyspace_group_manager_test.go | 2 +- pkg/utils/etcdutil/etcdutil_test.go | 14 ++--- pkg/utils/etcdutil/testutil.go | 16 +++--- pkg/utils/grpcutil/grpcutil.go | 3 +- pkg/utils/testutil/leak.go | 1 + pkg/utils/testutil/testutil.go | 3 +- server/config/config.go | 8 +-- server/server.go | 4 +- tests/integrations/go.mod | 11 ++-- tests/integrations/go.sum | 52 +++++-------------- tests/integrations/mcs/tso/proxy_test.go | 5 +- tests/integrations/mcs/tso/server_test.go | 3 +- tests/registry/registry_test.go | 6 +-- tools/go.mod | 11 ++-- tools/go.sum | 52 +++++-------------- tools/pd-simulator/simulator/client.go | 3 +- 25 files changed, 104 insertions(+), 183 deletions(-) diff --git a/client/grpcutil/grpcutil.go b/client/grpcutil/grpcutil.go index 742ee872b07..070cdf7822f 100644 --- a/client/grpcutil/grpcutil.go +++ b/client/grpcutil/grpcutil.go @@ -29,6 +29,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/backoff" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" ) @@ -54,7 +55,7 @@ const ( // ctx will be noop. Users should call ClientConn.Close to terminate all the // pending operations after this function returns. func GetClientConn(ctx context.Context, addr string, tlsCfg *tls.Config, do ...grpc.DialOption) (*grpc.ClientConn, error) { - opt := grpc.WithInsecure() //nolint + opt := grpc.WithTransportCredentials(insecure.NewCredentials()) if tlsCfg != nil { creds := credentials.NewTLS(tlsCfg) opt = grpc.WithTransportCredentials(creds) diff --git a/client/pd_service_discovery_test.go b/client/pd_service_discovery_test.go index d97f117790f..226d407b56b 100644 --- a/client/pd_service_discovery_test.go +++ b/client/pd_service_discovery_test.go @@ -32,6 +32,7 @@ import ( "github.com/tikv/pd/client/grpcutil" "github.com/tikv/pd/client/testutil" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" pb "google.golang.org/grpc/examples/helloworld/helloworld" "google.golang.org/grpc/health" healthpb "google.golang.org/grpc/health/grpc_health_v1" @@ -136,8 +137,8 @@ func (suite *serviceClientTestSuite) SetupSuite() { go suite.leaderServer.run() go suite.followerServer.run() for i := 0; i < 10; i++ { - leaderConn, err1 := grpc.Dial(suite.leaderServer.addr, grpc.WithInsecure()) //nolint - followerConn, err2 := grpc.Dial(suite.followerServer.addr, grpc.WithInsecure()) //nolint + leaderConn, err1 := grpc.Dial(suite.leaderServer.addr, grpc.WithTransportCredentials(insecure.NewCredentials())) + followerConn, err2 := grpc.Dial(suite.followerServer.addr, grpc.WithTransportCredentials(insecure.NewCredentials())) if err1 == nil && err2 == nil { suite.followerClient = newPDServiceClient(suite.followerServer.addr, suite.leaderServer.addr, nil, followerConn, false) suite.leaderClient = newPDServiceClient(suite.leaderServer.addr, suite.leaderServer.addr, nil, leaderConn, true) diff --git a/go.mod b/go.mod index 15934faff3a..baf14af152a 100644 --- a/go.mod +++ b/go.mod @@ -2,6 +2,11 @@ module github.com/tikv/pd go 1.21 +// When you modify PD cooperatively with kvproto, this will be useful to submit the PR to PD and the PR to +// kvproto at the same time. You can run `go mod tidy` to make it replaced with go-mod style specification. +// After the PR to kvproto is merged, remember to comment this out and run `go mod tidy`. +// replace github.com/pingcap/kvproto => github.com/$YourPrivateRepo $YourPrivateBranch + require ( github.com/AlekSi/gocov-xml v1.0.0 github.com/BurntSushi/toml v0.3.1 @@ -39,7 +44,7 @@ require ( github.com/sasha-s/go-deadlock v0.2.0 github.com/shirou/gopsutil/v3 v3.23.3 github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 - github.com/soheilhy/cmux v0.1.4 + github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.0.0 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.8.4 @@ -48,7 +53,7 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 github.com/unrolled/render v1.0.1 github.com/urfave/negroni v0.3.0 - go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 + go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b go.uber.org/atomic v1.10.0 go.uber.org/goleak v1.2.0 go.uber.org/zap v1.26.0 @@ -114,7 +119,7 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/pprof v0.0.0-20211122183932-1daafda22083 // indirect github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.3.1 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 // indirect github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect @@ -173,7 +178,7 @@ require ( github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect // Fix panic in unit test with go >= 1.14, ref: etcd-io/bbolt#201 https://github.com/etcd-io/bbolt/pull/201 - go.etcd.io/bbolt v1.3.6 // indirect + go.etcd.io/bbolt v1.3.8 // indirect go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect go.uber.org/multierr v1.11.0 // indirect @@ -202,10 +207,3 @@ require ( moul.io/zapgorm2 v1.1.0 // indirect sigs.k8s.io/yaml v1.2.0 // indirect ) - -replace google.golang.org/grpc v1.59.0 => google.golang.org/grpc v1.26.0 - -// When you modify PD cooperatively with kvproto, this will be useful to submit the PR to PD and the PR to -// kvproto at the same time. You can run `go mod tidy` to make it replaced with go-mod style specification. -// After the PR to kvproto is merged, remember to comment this out and run `go mod tidy`. -// replace github.com/pingcap/kvproto => github.com/$YourPrivateRepo $YourPrivateBranch diff --git a/go.sum b/go.sum index 9debe93e252..650e972ee40 100644 --- a/go.sum +++ b/go.sum @@ -60,7 +60,6 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= 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/bitly/go-simplejson v0.5.0 h1:6IH+V8/tVMab511d5bn4M7EwGXZf9Hj6i2xSwkNEM+Y= github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= @@ -95,11 +94,9 @@ github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -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/corona10/goimagehash v1.0.2 h1:pUfB0LnsJASMPGEZLj7tGY251vF+qLGqOgEP4rUs6kA= @@ -108,7 +105,6 @@ github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:ma github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= 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= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -118,7 +114,6 @@ github.com/dnephin/pflag v1.0.7 h1:oxONGlWxhmUct0YzKTgrpQv9AUA1wtPBn7zuSjJqptk= github.com/dnephin/pflag v1.0.7/go.mod h1:uxE91IoWURlOiTUIA8Mq5ZZkAv3dPUfZNaT80Zm7OQE= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= @@ -129,7 +124,6 @@ github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymF github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= 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/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= @@ -214,9 +208,6 @@ github.com/golang-sql/sqlexp v0.1.0/go.mod h1:J4ad9Vo8ZCWQ2GMrC4UCQy1JpCbwU9m3EO github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= -github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef h1:veQD95Isof8w9/WXiA+pa3tz3fJXkt5B7QaRBrM62gk= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -248,10 +239,9 @@ github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8I github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= -github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.7.4 h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= @@ -264,7 +254,6 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69 h1:7xsUJsB2NrdcttQPa7JLEaGzvdbk7KvfrjgHZXOQRo0= @@ -315,8 +304,6 @@ github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFF github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= @@ -353,7 +340,6 @@ github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.7.6 h1:8yTIVnZgCoiM1TgqoeTl+LfU5Jg6/xL3QhGQnimLYnA= github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.8 h1:c1ghPdyEDarC70ftn0y+A/Ee++9zz8ljHG1b13eJ0s8= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= @@ -364,7 +350,6 @@ github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Ky 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= github.com/mattn/go-runewidth v0.0.8/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= @@ -398,7 +383,6 @@ github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLA github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1 h1:D1XO3LVEYroYskEsoSiGItp9RUxG6jWnCVvrqH0HHQM= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.4 h1:vHD/YYe1Wolo78koG299f7V/VAS08c6IpCLn+Ejf/w8= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -456,7 +440,6 @@ github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3g github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= @@ -466,12 +449,10 @@ github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cY github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= @@ -505,23 +486,21 @@ github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919Lwc github.com/shurcooL/sanitized_anchor_name v1.0.0 h1:PdmoCO6wvbs+7yrJyMORt4/BmY5IYyJwS/kOiWx8mHo= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 h1:Txo4SXVJq/OgEjwgkWoxkMoTjGlcrgsQE/XSghjmu0w= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072/go.mod h1:+4nWMF0+CqEcU74SnX2NxaGqZ8zX4pcQ8Jcs77DbX5A= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= @@ -583,7 +562,6 @@ github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4d 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= github.com/urfave/cli/v2 v2.3.0 h1:qph92Y649prgesehzOrQjdWyxFOp/QVM+6imKHad91M= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= @@ -602,10 +580,10 @@ github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5t github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= -go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 h1:fqmtdYQlwZ/vKWSz5amW+a4cnjg23ojz5iL7rjf08Wg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793/go.mod h1:eBhtbxXP1qpW0F6+WxoJ64DM1Mrfx46PHtVxEdkLe0I= +go.etcd.io/bbolt v1.3.8 h1:xs88BrvEv273UsB79e0hcVrlUWmS0a8upikMFhSyAtA= +go.etcd.io/bbolt v1.3.8/go.mod h1:N9Mkw9X8x5fupy0IKsmuqVtoGDyxsaDlbk4Rd05IAQw= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b h1:NeKdMkWVeCdzdg9Rip/aVOgBeoGAjldAZ2pXLskuksU= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b/go.mod h1:WnaHjGRsd7pp+qO+AuA+fyHoBefpgyaTpSZ3Qw4H1aY= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -650,7 +628,6 @@ golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -691,11 +668,11 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL 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-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/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-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= @@ -731,7 +708,6 @@ golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -746,7 +722,6 @@ golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-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= @@ -769,7 +744,6 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -826,10 +800,10 @@ google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ij google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= @@ -842,7 +816,6 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -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= @@ -893,6 +866,5 @@ 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/dashboard/adapter/config.go b/pkg/dashboard/adapter/config.go index a1661b84f2b..348b146c854 100644 --- a/pkg/dashboard/adapter/config.go +++ b/pkg/dashboard/adapter/config.go @@ -31,7 +31,7 @@ func GenDashboardConfig(srv *server.Server) (*config.Config, error) { dashboardCfg := config.Default() dashboardCfg.DataDir = cfg.DataDir - dashboardCfg.PDEndPoint = etcdCfg.ACUrls[0].String() + dashboardCfg.PDEndPoint = etcdCfg.AdvertiseClientUrls[0].String() dashboardCfg.PublicPathPrefix = cfg.Dashboard.PublicPathPrefix dashboardCfg.EnableTelemetry = cfg.Dashboard.EnableTelemetry dashboardCfg.EnableExperimental = cfg.Dashboard.EnableExperimental diff --git a/pkg/election/leadership_test.go b/pkg/election/leadership_test.go index be1922fe381..de2e4b1129b 100644 --- a/pkg/election/leadership_test.go +++ b/pkg/election/leadership_test.go @@ -155,7 +155,7 @@ func TestExitWatch(t *testing.T) { checkExitWatch(t, leaderKey, func(server *embed.Etcd, client *clientv3.Client) func() { cfg1 := server.Config() etcd2 := etcdutil.MustAddEtcdMember(t, &cfg1, client) - client2, err := etcdutil.CreateEtcdClient(nil, etcd2.Config().LCUrls) + client2, err := etcdutil.CreateEtcdClient(nil, etcd2.Config().ListenClientUrls) re.NoError(err) // close the original leader server.Server.HardStop() @@ -189,7 +189,7 @@ func checkExitWatch(t *testing.T, leaderKey string, injectFunc func(server *embe re := require.New(t) servers, client1, clean := etcdutil.NewTestEtcdCluster(t, 1) defer clean() - client2, err := etcdutil.CreateEtcdClient(nil, servers[0].Config().LCUrls) + client2, err := etcdutil.CreateEtcdClient(nil, servers[0].Config().ListenClientUrls) re.NoError(err) defer client2.Close() @@ -225,7 +225,7 @@ func TestRequestProgress(t *testing.T) { defer os.RemoveAll(fname) servers, client1, clean := etcdutil.NewTestEtcdCluster(t, 1) defer clean() - client2, err := etcdutil.CreateEtcdClient(nil, servers[0].Config().LCUrls) + client2, err := etcdutil.CreateEtcdClient(nil, servers[0].Config().ListenClientUrls) re.NoError(err) defer client2.Close() diff --git a/pkg/mcs/scheduling/server/rule/watcher_test.go b/pkg/mcs/scheduling/server/rule/watcher_test.go index dafc7dcac2f..37fce0a0ded 100644 --- a/pkg/mcs/scheduling/server/rule/watcher_test.go +++ b/pkg/mcs/scheduling/server/rule/watcher_test.go @@ -86,7 +86,7 @@ func prepare(t require.TestingT) (context.Context, *clientv3.Client, func()) { os.RemoveAll(cfg.Dir) etcd, err := embed.StartEtcd(cfg) re.NoError(err) - client, err := etcdutil.CreateEtcdClient(nil, cfg.LCUrls) + client, err := etcdutil.CreateEtcdClient(nil, cfg.ListenClientUrls) re.NoError(err) <-etcd.Server.ReadyNotify() diff --git a/pkg/mcs/tso/server/testutil.go b/pkg/mcs/tso/server/testutil.go index 626d1474673..cf5d45e7754 100644 --- a/pkg/mcs/tso/server/testutil.go +++ b/pkg/mcs/tso/server/testutil.go @@ -21,11 +21,12 @@ import ( "github.com/spf13/pflag" "github.com/stretchr/testify/require" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" ) // MustNewGrpcClient must create a new TSO grpc client. func MustNewGrpcClient(re *require.Assertions, addr string) (*grpc.ClientConn, tsopb.TSOClient) { - conn, err := grpc.Dial(strings.TrimPrefix(addr, "http://"), grpc.WithInsecure()) + conn, err := grpc.Dial(strings.TrimPrefix(addr, "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) return conn, tsopb.NewTSOClient(conn) } diff --git a/pkg/syncer/client_test.go b/pkg/syncer/client_test.go index 6770fae44ac..84193ebaffe 100644 --- a/pkg/syncer/client_test.go +++ b/pkg/syncer/client_test.go @@ -71,7 +71,7 @@ func TestErrorCode(t *testing.T) { } ctx, cancel := context.WithCancel(context.TODO()) rc := NewRegionSyncer(server) - conn, err := grpcutil.GetClientConn(ctx, "127.0.0.1", nil) + conn, err := grpcutil.GetClientConn(ctx, "http://127.0.0.1", nil) re.NoError(err) cancel() _, err = rc.syncRegion(ctx, conn) diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 7af76ee6c28..9fe6da6edc9 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -69,7 +69,7 @@ func (suite *keyspaceGroupManagerTestSuite) SetupSuite() { suite.ctx, suite.cancel = context.WithCancel(context.Background()) suite.ClusterID = rand.Uint64() servers, client, clean := etcdutil.NewTestEtcdCluster(t, 1) - suite.backendEndpoints, suite.etcdClient, suite.clean = servers[0].Config().LCUrls[0].String(), client, clean + suite.backendEndpoints, suite.etcdClient, suite.clean = servers[0].Config().ListenClientUrls[0].String(), client, clean suite.cfg = suite.createConfig() } diff --git a/pkg/utils/etcdutil/etcdutil_test.go b/pkg/utils/etcdutil/etcdutil_test.go index 1e5c8a3316f..63fa50fd800 100644 --- a/pkg/utils/etcdutil/etcdutil_test.go +++ b/pkg/utils/etcdutil/etcdutil_test.go @@ -211,10 +211,10 @@ func TestEtcdScaleInAndOut(t *testing.T) { etcd1, cfg1 := servers[0], servers[0].Config() // Create two etcd clients with etcd1 as endpoint. - client1, err := CreateEtcdClient(nil, cfg1.LCUrls) // execute member change operation with this client + client1, err := CreateEtcdClient(nil, cfg1.ListenClientUrls) // execute member change operation with this client re.NoError(err) defer client1.Close() - client2, err := CreateEtcdClient(nil, cfg1.LCUrls) // check member change with this client + client2, err := CreateEtcdClient(nil, cfg1.ListenClientUrls) // check member change with this client re.NoError(err) defer client2.Close() @@ -287,7 +287,7 @@ func checkEtcdWithHangLeader(t *testing.T) error { var enableDiscard atomic.Bool ctx, cancel := context.WithCancel(context.Background()) defer cancel() - go proxyWithDiscard(ctx, re, cfg1.LCUrls[0].String(), proxyAddr, &enableDiscard) + go proxyWithDiscard(ctx, re, cfg1.ListenClientUrls[0].String(), proxyAddr, &enableDiscard) // Create an etcd client with etcd1 as endpoint. urls, err := types.NewURLs([]string{proxyAddr}) @@ -409,7 +409,7 @@ func (suite *loopWatcherTestSuite) SetupSuite() { suite.config = NewTestSingleConfig() suite.config.Dir = suite.T().TempDir() suite.startEtcd(re) - suite.client, err = CreateEtcdClient(nil, suite.config.LCUrls) + suite.client, err = CreateEtcdClient(nil, suite.config.ListenClientUrls) re.NoError(err) suite.cleans = append(suite.cleans, func() { suite.client.Close() @@ -668,7 +668,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { // Case2: close the etcd client and put a new value after watcher restarts suite.client.Close() - suite.client, err = CreateEtcdClient(nil, suite.config.LCUrls) + suite.client, err = CreateEtcdClient(nil, suite.config.ListenClientUrls) re.NoError(err) watcher.updateClientCh <- suite.client suite.put(re, "TestWatcherBreak", "2") @@ -676,7 +676,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { // Case3: close the etcd client and put a new value before watcher restarts suite.client.Close() - suite.client, err = CreateEtcdClient(nil, suite.config.LCUrls) + suite.client, err = CreateEtcdClient(nil, suite.config.ListenClientUrls) re.NoError(err) suite.put(re, "TestWatcherBreak", "3") watcher.updateClientCh <- suite.client @@ -684,7 +684,7 @@ func (suite *loopWatcherTestSuite) TestWatcherBreak() { // Case4: close the etcd client and put a new value with compact suite.client.Close() - suite.client, err = CreateEtcdClient(nil, suite.config.LCUrls) + suite.client, err = CreateEtcdClient(nil, suite.config.ListenClientUrls) re.NoError(err) suite.put(re, "TestWatcherBreak", "4") resp, err := EtcdKVGet(suite.client, "TestWatcherBreak") diff --git a/pkg/utils/etcdutil/testutil.go b/pkg/utils/etcdutil/testutil.go index 57f7200ecb8..3ea4d057645 100644 --- a/pkg/utils/etcdutil/testutil.go +++ b/pkg/utils/etcdutil/testutil.go @@ -38,14 +38,14 @@ func NewTestSingleConfig() *embed.Config { cfg.LogOutputs = []string{"stdout"} pu, _ := url.Parse(tempurl.Alloc()) - cfg.LPUrls = []url.URL{*pu} - cfg.APUrls = cfg.LPUrls + cfg.ListenPeerUrls = []url.URL{*pu} + cfg.AdvertisePeerUrls = cfg.ListenPeerUrls cu, _ := url.Parse(tempurl.Alloc()) - cfg.LCUrls = []url.URL{*cu} - cfg.ACUrls = cfg.LCUrls + cfg.ListenClientUrls = []url.URL{*cu} + cfg.AdvertiseClientUrls = cfg.ListenClientUrls cfg.StrictReconfigCheck = false - cfg.InitialCluster = fmt.Sprintf("%s=%s", cfg.Name, &cfg.LPUrls[0]) + cfg.InitialCluster = fmt.Sprintf("%s=%s", cfg.Name, &cfg.ListenPeerUrls[0]) cfg.ClusterState = embed.ClusterStateFlagNew return cfg } @@ -63,7 +63,7 @@ func NewTestEtcdCluster(t *testing.T, count int) (servers []*embed.Etcd, etcdCli cfg.Dir = t.TempDir() etcd, err := embed.StartEtcd(cfg) re.NoError(err) - etcdClient, err = CreateEtcdClient(nil, cfg.LCUrls) + etcdClient, err = CreateEtcdClient(nil, cfg.ListenClientUrls) re.NoError(err) <-etcd.Server.ReadyNotify() servers = append(servers, etcd) @@ -101,9 +101,9 @@ func MustAddEtcdMember(t *testing.T, cfg1 *embed.Config, client *clientv3.Client cfg2 := NewTestSingleConfig() cfg2.Dir = t.TempDir() cfg2.Name = genRandName() - cfg2.InitialCluster = cfg1.InitialCluster + fmt.Sprintf(",%s=%s", cfg2.Name, &cfg2.LPUrls[0]) + cfg2.InitialCluster = cfg1.InitialCluster + fmt.Sprintf(",%s=%s", cfg2.Name, &cfg2.ListenPeerUrls[0]) cfg2.ClusterState = embed.ClusterStateFlagExisting - peerURL := cfg2.LPUrls[0].String() + peerURL := cfg2.ListenPeerUrls[0].String() addResp, err := AddEtcdMember(client, []string{peerURL}) re.NoError(err) // Check the client can get the new member. diff --git a/pkg/utils/grpcutil/grpcutil.go b/pkg/utils/grpcutil/grpcutil.go index d108a6f5d44..3b233956fa8 100644 --- a/pkg/utils/grpcutil/grpcutil.go +++ b/pkg/utils/grpcutil/grpcutil.go @@ -33,6 +33,7 @@ import ( "google.golang.org/grpc/backoff" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/metadata" ) @@ -139,7 +140,7 @@ func (s TLSConfig) GetOneAllowedCN() (string, error) { // ctx will be noop. Users should call ClientConn.Close to terminate all the // pending operations after this function returns. func GetClientConn(ctx context.Context, addr string, tlsCfg *tls.Config, do ...grpc.DialOption) (*grpc.ClientConn, error) { - opt := grpc.WithInsecure() + opt := grpc.WithTransportCredentials(insecure.NewCredentials()) if tlsCfg != nil { creds := credentials.NewTLS(tlsCfg) opt = grpc.WithTransportCredentials(creds) diff --git a/pkg/utils/testutil/leak.go b/pkg/utils/testutil/leak.go index d1329aef0e6..ba2ebb7fcb0 100644 --- a/pkg/utils/testutil/leak.go +++ b/pkg/utils/testutil/leak.go @@ -21,6 +21,7 @@ var LeakOptions = []goleak.Option{ goleak.IgnoreTopFunction("github.com/syndtr/goleveldb/leveldb.(*DB).mpoolDrain"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), + goleak.IgnoreTopFunction("google.golang.org/grpc/internal/grpcsync.(*CallbackSerializer).run"), goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("sync.runtime_notifyListWait"), // TODO: remove the below options once we fixed the http connection leak problems diff --git a/pkg/utils/testutil/testutil.go b/pkg/utils/testutil/testutil.go index a48db0bd60f..a41fc436ca6 100644 --- a/pkg/utils/testutil/testutil.go +++ b/pkg/utils/testutil/testutil.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/log" "github.com/stretchr/testify/require" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" ) const ( @@ -77,7 +78,7 @@ func NewRequestHeader(clusterID uint64) *pdpb.RequestHeader { // MustNewGrpcClient must create a new PD grpc client. func MustNewGrpcClient(re *require.Assertions, addr string) pdpb.PDClient { - conn, err := grpc.Dial(strings.TrimPrefix(addr, "http://"), grpc.WithInsecure()) + conn, err := grpc.Dial(strings.TrimPrefix(addr, "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) return pdpb.NewPDClient(conn) } diff --git a/server/config/config.go b/server/config/config.go index 25e13d59652..6be949e8a62 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -748,22 +748,22 @@ func (c *Config) GenEmbedEtcdConfig() (*embed.Config, error) { cfg.Logger = "zap" var err error - cfg.LPUrls, err = parseUrls(c.PeerUrls) + cfg.ListenPeerUrls, err = parseUrls(c.PeerUrls) if err != nil { return nil, err } - cfg.APUrls, err = parseUrls(c.AdvertisePeerUrls) + cfg.AdvertisePeerUrls, err = parseUrls(c.AdvertisePeerUrls) if err != nil { return nil, err } - cfg.LCUrls, err = parseUrls(c.ClientUrls) + cfg.ListenClientUrls, err = parseUrls(c.ClientUrls) if err != nil { return nil, err } - cfg.ACUrls, err = parseUrls(c.AdvertiseClientUrls) + cfg.AdvertiseClientUrls, err = parseUrls(c.AdvertiseClientUrls) if err != nil { return nil, err } diff --git a/server/server.go b/server/server.go index 4c1632f634a..6e9341298cd 100644 --- a/server/server.go +++ b/server/server.go @@ -384,12 +384,12 @@ func (s *Server) startClient() error { } /* Starting two different etcd clients here is to avoid the throttling. */ // This etcd client will be used to access the etcd cluster to read and write all kinds of meta data. - s.client, err = etcdutil.CreateEtcdClient(tlsConfig, etcdCfg.ACUrls, "server-etcd-client") + s.client, err = etcdutil.CreateEtcdClient(tlsConfig, etcdCfg.AdvertiseClientUrls, "server-etcd-client") if err != nil { return errs.ErrNewEtcdClient.Wrap(err).GenWithStackByCause() } // This etcd client will only be used to read and write the election-related data, such as leader key. - s.electionClient, err = etcdutil.CreateEtcdClient(tlsConfig, etcdCfg.ACUrls, "election-etcd-client") + s.electionClient, err = etcdutil.CreateEtcdClient(tlsConfig, etcdCfg.AdvertiseClientUrls, "election-etcd-client") if err != nil { return errs.ErrNewEtcdClient.Wrap(err).GenWithStackByCause() } diff --git a/tests/integrations/go.mod b/tests/integrations/go.mod index 2f6e7e71eb5..9a3270e44de 100644 --- a/tests/integrations/go.mod +++ b/tests/integrations/go.mod @@ -8,9 +8,6 @@ replace ( github.com/tikv/pd/tests/integrations/mcs => ./mcs ) -// reset grpc and protobuf deps in order to import client and server at the same time -replace google.golang.org/grpc v1.59.0 => google.golang.org/grpc v1.26.0 - require ( github.com/DATA-DOG/go-sqlmock v1.5.0 github.com/docker/go-units v0.5.0 @@ -24,7 +21,7 @@ require ( github.com/stretchr/testify v1.8.4 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 - go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 + go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b go.uber.org/goleak v1.3.0 go.uber.org/zap v1.26.0 google.golang.org/grpc v1.59.0 @@ -95,7 +92,7 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/btree v1.1.2 // indirect github.com/google/pprof v0.0.0-20211122183932-1daafda22083 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.3.1 // indirect github.com/gorilla/mux v1.7.4 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 // indirect @@ -144,7 +141,7 @@ require ( github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect github.com/sirupsen/logrus v1.6.0 // indirect github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 // indirect - github.com/soheilhy/cmux v0.1.4 // indirect + github.com/soheilhy/cmux v0.1.5 // indirect github.com/spf13/cobra v1.0.0 // indirect github.com/spf13/pflag v1.0.5 // indirect github.com/stretchr/objx v0.5.0 // indirect @@ -163,7 +160,7 @@ require ( github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect - go.etcd.io/bbolt v1.3.6 // indirect + go.etcd.io/bbolt v1.3.8 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect diff --git a/tests/integrations/go.sum b/tests/integrations/go.sum index a1c9fcccf91..5e128995307 100644 --- a/tests/integrations/go.sum +++ b/tests/integrations/go.sum @@ -58,7 +58,6 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= 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/bitly/go-simplejson v0.5.0 h1:6IH+V8/tVMab511d5bn4M7EwGXZf9Hj6i2xSwkNEM+Y= github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= @@ -95,11 +94,9 @@ github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -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/corona10/goimagehash v1.0.2 h1:pUfB0LnsJASMPGEZLj7tGY251vF+qLGqOgEP4rUs6kA= @@ -107,7 +104,6 @@ github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawk github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= 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= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -115,7 +111,6 @@ github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZm github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= @@ -126,7 +121,6 @@ github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymF github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= 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/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= @@ -205,9 +199,6 @@ github.com/golang-sql/sqlexp v0.1.0/go.mod h1:J4ad9Vo8ZCWQ2GMrC4UCQy1JpCbwU9m3EO github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= -github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef h1:veQD95Isof8w9/WXiA+pa3tz3fJXkt5B7QaRBrM62gk= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -237,10 +228,9 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= -github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.7.4 h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= @@ -253,7 +243,6 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69 h1:7xsUJsB2NrdcttQPa7JLEaGzvdbk7KvfrjgHZXOQRo0= @@ -303,8 +292,6 @@ github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFF github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= @@ -342,13 +329,11 @@ github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.7.6 h1:8yTIVnZgCoiM1TgqoeTl+LfU5Jg6/xL3QhGQnimLYnA= github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= -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/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= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -376,7 +361,6 @@ github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+ github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1 h1:D1XO3LVEYroYskEsoSiGItp9RUxG6jWnCVvrqH0HHQM= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= @@ -438,7 +422,6 @@ github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3g github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= @@ -448,12 +431,10 @@ github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cY github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= @@ -484,23 +465,21 @@ github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 h1:mj/nMDAwTBiaC github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 h1:Txo4SXVJq/OgEjwgkWoxkMoTjGlcrgsQE/XSghjmu0w= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072/go.mod h1:+4nWMF0+CqEcU74SnX2NxaGqZ8zX4pcQ8Jcs77DbX5A= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= @@ -560,7 +539,6 @@ github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4d 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= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= @@ -578,10 +556,10 @@ github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5t github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= -go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 h1:fqmtdYQlwZ/vKWSz5amW+a4cnjg23ojz5iL7rjf08Wg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793/go.mod h1:eBhtbxXP1qpW0F6+WxoJ64DM1Mrfx46PHtVxEdkLe0I= +go.etcd.io/bbolt v1.3.8 h1:xs88BrvEv273UsB79e0hcVrlUWmS0a8upikMFhSyAtA= +go.etcd.io/bbolt v1.3.8/go.mod h1:N9Mkw9X8x5fupy0IKsmuqVtoGDyxsaDlbk4Rd05IAQw= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b h1:NeKdMkWVeCdzdg9Rip/aVOgBeoGAjldAZ2pXLskuksU= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b/go.mod h1:WnaHjGRsd7pp+qO+AuA+fyHoBefpgyaTpSZ3Qw4H1aY= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -626,7 +604,6 @@ golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -669,11 +646,11 @@ golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/ 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-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/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-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= @@ -705,7 +682,6 @@ golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -719,7 +695,6 @@ golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-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= @@ -740,7 +715,6 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -794,10 +768,12 @@ google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ij google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= @@ -810,7 +786,6 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -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/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= @@ -855,6 +830,5 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt 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/tso/proxy_test.go b/tests/integrations/mcs/tso/proxy_test.go index a19776ffc48..7ed329610f2 100644 --- a/tests/integrations/mcs/tso/proxy_test.go +++ b/tests/integrations/mcs/tso/proxy_test.go @@ -34,6 +34,7 @@ import ( "github.com/tikv/pd/tests" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" ) type tsoProxyTestSuite struct { @@ -177,7 +178,7 @@ func (s *tsoProxyTestSuite) TestTSOProxyClientsWithSameContext() { defer cancel() for i := 0; i < clientCount; i++ { - conn, err := grpc.Dial(strings.TrimPrefix(s.backendEndpoints, "http://"), grpc.WithInsecure()) + conn, err := grpc.Dial(strings.TrimPrefix(s.backendEndpoints, "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) grpcPDClient := pdpb.NewPDClient(conn) stream, err := grpcPDClient.Tso(ctx) @@ -375,7 +376,7 @@ func createTSOStreams( streams := make([]pdpb.PD_TsoClient, clientCount) for i := 0; i < clientCount; i++ { - conn, err := grpc.Dial(strings.TrimPrefix(backendEndpoints, "http://"), grpc.WithInsecure()) + conn, err := grpc.Dial(strings.TrimPrefix(backendEndpoints, "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) grpcPDClient := pdpb.NewPDClient(conn) cctx, cancel := context.WithCancel(ctx) diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index b175f63c8f4..108740e46f9 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -45,6 +45,7 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/goleak" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" ) func TestMain(m *testing.M) { @@ -103,7 +104,7 @@ func (suite *tsoServerTestSuite) TestTSOServerStartAndStopNormally() { }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) // Test registered GRPC Service - cc, err := grpc.DialContext(suite.ctx, s.GetAddr(), grpc.WithInsecure()) + cc, err := grpc.DialContext(suite.ctx, s.GetAddr(), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) cc.Close() diff --git a/tests/registry/registry_test.go b/tests/registry/registry_test.go index d2661cda616..dab2ccae683 100644 --- a/tests/registry/registry_test.go +++ b/tests/registry/registry_test.go @@ -30,8 +30,8 @@ import ( "github.com/tikv/pd/tests" "go.uber.org/goleak" "google.golang.org/grpc" - - "google.golang.org/grpc/test/grpc_testing" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/interop/grpc_testing" ) func TestMain(m *testing.M) { @@ -80,7 +80,7 @@ func TestRegistryService(t *testing.T) { leader := cluster.GetLeaderServer() // Test registered GRPC Service - cc, err := grpc.DialContext(ctx, strings.TrimPrefix(leader.GetAddr(), "http://"), grpc.WithInsecure()) + cc, err := grpc.DialContext(ctx, strings.TrimPrefix(leader.GetAddr(), "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) re.NoError(err) defer cc.Close() grpcClient := grpc_testing.NewTestServiceClient(cc) diff --git a/tools/go.mod b/tools/go.mod index 3e63141a89b..c5cf791d5c0 100644 --- a/tools/go.mod +++ b/tools/go.mod @@ -7,9 +7,6 @@ replace ( github.com/tikv/pd/client => ../client ) -// reset grpc and protobuf deps in order to import client and server at the same time -replace google.golang.org/grpc v1.59.0 => google.golang.org/grpc v1.26.0 - require ( github.com/BurntSushi/toml v0.3.1 github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e @@ -33,7 +30,7 @@ require ( github.com/stretchr/testify v1.8.4 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 - go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 + go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b go.uber.org/goleak v1.2.0 go.uber.org/zap v1.26.0 golang.org/x/text v0.14.0 @@ -96,7 +93,7 @@ require ( github.com/golang/snappy v0.0.4 // indirect github.com/google/btree v1.1.2 // indirect github.com/google/pprof v0.0.0-20211122183932-1daafda22083 // indirect - github.com/google/uuid v1.3.0 // indirect + github.com/google/uuid v1.3.1 // indirect github.com/gorilla/mux v1.7.4 // indirect github.com/gorilla/websocket v1.4.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 // indirect @@ -144,7 +141,7 @@ require ( github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 // indirect github.com/sirupsen/logrus v1.6.0 // indirect github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 // indirect - github.com/soheilhy/cmux v0.1.4 // indirect + github.com/soheilhy/cmux v0.1.5 // indirect github.com/stretchr/objx v0.5.0 // indirect github.com/swaggo/files v0.0.0-20210815190702-a29dd2bc99b2 // indirect github.com/swaggo/http-swagger v1.2.6 // indirect @@ -161,7 +158,7 @@ require ( github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect - go.etcd.io/bbolt v1.3.6 // indirect + go.etcd.io/bbolt v1.3.8 // indirect go.uber.org/atomic v1.10.0 // indirect go.uber.org/dig v1.9.0 // indirect go.uber.org/fx v1.12.0 // indirect diff --git a/tools/go.sum b/tools/go.sum index 8711bc3f8aa..3e19c7fedcf 100644 --- a/tools/go.sum +++ b/tools/go.sum @@ -56,7 +56,6 @@ github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24 github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= 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/bitly/go-simplejson v0.5.0 h1:6IH+V8/tVMab511d5bn4M7EwGXZf9Hj6i2xSwkNEM+Y= github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA= github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869 h1:DDGfHa7BWjL4YnC6+E63dPcxHo2sUxDIu8g3QgEJdRY= @@ -94,11 +93,9 @@ github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -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/corona10/goimagehash v1.0.2 h1:pUfB0LnsJASMPGEZLj7tGY251vF+qLGqOgEP4rUs6kA= @@ -106,7 +103,6 @@ github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawk github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= 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= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -114,7 +110,6 @@ github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZm github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= @@ -125,7 +120,6 @@ github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymF github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= 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/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= @@ -203,9 +197,6 @@ github.com/golang-sql/sqlexp v0.1.0/go.mod h1:J4ad9Vo8ZCWQ2GMrC4UCQy1JpCbwU9m3EO github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 h1:DACJavvAHhabrF08vX0COfcOBJRhZ8lUbR+ZWIs0Y5g= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= -github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef h1:veQD95Isof8w9/WXiA+pa3tz3fJXkt5B7QaRBrM62gk= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= @@ -235,10 +226,9 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/ github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= -github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= +github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/mux v1.7.4 h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= @@ -251,7 +241,6 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69 h1:7xsUJsB2NrdcttQPa7JLEaGzvdbk7KvfrjgHZXOQRo0= @@ -304,8 +293,6 @@ github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFF github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= @@ -342,13 +329,11 @@ github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.7.6 h1:8yTIVnZgCoiM1TgqoeTl+LfU5Jg6/xL3QhGQnimLYnA= github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= -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/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-shellwords v1.0.12 h1:M2zGm7EW6UQJvDeQxo4T51eKPurbeFbe8WtebGE2xrk= github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= github.com/mattn/go-sqlite3 v1.14.15 h1:vfoHhTN1af61xCRSWzFIWzx2YskyMTwHLrExkBOjvxI= @@ -376,7 +361,6 @@ github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLA github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1 h1:D1XO3LVEYroYskEsoSiGItp9RUxG6jWnCVvrqH0HHQM= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -434,7 +418,6 @@ github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3g github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= @@ -444,12 +427,10 @@ github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cY github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= @@ -480,23 +461,21 @@ github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 h1:mj/nMDAwTBiaC github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0 h1:UBcNElsrwanuuMsnGSlYmtmgbb23qDR5dG+6X6Oo89I= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072 h1:Txo4SXVJq/OgEjwgkWoxkMoTjGlcrgsQE/XSghjmu0w= github.com/smallnest/chanx v0.0.0-20221229104322-eb4c998d2072/go.mod h1:+4nWMF0+CqEcU74SnX2NxaGqZ8zX4pcQ8Jcs77DbX5A= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= @@ -557,7 +536,6 @@ github.com/ugorji/go/codec v1.2.11 h1:BMaWp1Bb6fHwEtbplGBGJ498wD+LKlNSl25MjdZY4d 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= github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= @@ -575,10 +553,10 @@ github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5t github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= -go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793 h1:fqmtdYQlwZ/vKWSz5amW+a4cnjg23ojz5iL7rjf08Wg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20220915004622-85b640cee793/go.mod h1:eBhtbxXP1qpW0F6+WxoJ64DM1Mrfx46PHtVxEdkLe0I= +go.etcd.io/bbolt v1.3.8 h1:xs88BrvEv273UsB79e0hcVrlUWmS0a8upikMFhSyAtA= +go.etcd.io/bbolt v1.3.8/go.mod h1:N9Mkw9X8x5fupy0IKsmuqVtoGDyxsaDlbk4Rd05IAQw= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b h1:NeKdMkWVeCdzdg9Rip/aVOgBeoGAjldAZ2pXLskuksU= +go.etcd.io/etcd v0.5.0-alpha.5.0.20240131130919-ff2304879e1b/go.mod h1:WnaHjGRsd7pp+qO+AuA+fyHoBefpgyaTpSZ3Qw4H1aY= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -623,7 +601,6 @@ golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210711020723-a769d52b0f97/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -665,11 +642,11 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL 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-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/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-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= @@ -701,7 +678,6 @@ golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -715,7 +691,6 @@ golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-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= @@ -736,7 +711,6 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -793,10 +767,12 @@ google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ij google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= @@ -809,7 +785,6 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -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= @@ -855,6 +830,5 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt 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/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index b461bffebb0..81453307afa 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -33,6 +33,7 @@ import ( "github.com/tikv/pd/tools/pd-simulator/simulator/simutil" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" ) // Client is a PD (Placement Driver) client. @@ -135,7 +136,7 @@ func (c *client) getMembers(ctx context.Context) (*pdpb.GetMembersResponse, erro } func (c *client) createConn() (*grpc.ClientConn, error) { - cc, err := grpc.Dial(strings.TrimPrefix(c.url, "http://"), grpc.WithInsecure()) + cc, err := grpc.Dial(strings.TrimPrefix(c.url, "http://"), grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { return nil, errors.WithStack(err) } From bbd3bdb56f80faddda222cb63628c5007fd330bd Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Thu, 7 Mar 2024 14:33:36 +0800 Subject: [PATCH 10/10] scheduler: make history-sample-interval and history-sample-duration configurable (#7878) close tikv/pd#7877 Signed-off-by: lhy1024 --- pkg/schedule/schedulers/grant_hot_region.go | 3 +- pkg/schedule/schedulers/hot_region.go | 14 +++- pkg/schedule/schedulers/hot_region_config.go | 33 ++++++++ pkg/schedule/schedulers/hot_region_test.go | 22 +++++- pkg/schedule/schedulers/hot_region_v2_test.go | 9 ++- pkg/schedule/schedulers/shuffle_hot_region.go | 3 +- pkg/statistics/store_collection_test.go | 4 +- pkg/statistics/store_load.go | 75 ++++++++++++------- pkg/statistics/store_load_test.go | 20 ++++- tests/server/api/scheduler_test.go | 2 + .../pd-ctl/tests/scheduler/scheduler_test.go | 22 ++++++ 11 files changed, 164 insertions(+), 43 deletions(-) diff --git a/pkg/schedule/schedulers/grant_hot_region.go b/pkg/schedule/schedulers/grant_hot_region.go index 37d1707339e..262dfe73873 100644 --- a/pkg/schedule/schedulers/grant_hot_region.go +++ b/pkg/schedule/schedulers/grant_hot_region.go @@ -137,7 +137,8 @@ type grantHotRegionScheduler struct { // newGrantHotRegionScheduler creates an admin scheduler that transfers hot region peer to fixed store and hot region leader to one store. func newGrantHotRegionScheduler(opController *operator.Controller, conf *grantHotRegionSchedulerConfig) *grantHotRegionScheduler { - base := newBaseHotScheduler(opController) + base := newBaseHotScheduler(opController, + statistics.DefaultHistorySampleDuration, statistics.DefaultHistorySampleInterval) handler := newGrantHotRegionHandler(conf) ret := &grantHotRegionScheduler{ baseHotScheduler: base, diff --git a/pkg/schedule/schedulers/hot_region.go b/pkg/schedule/schedulers/hot_region.go index fdd07e85145..2a38ef399c8 100644 --- a/pkg/schedule/schedulers/hot_region.go +++ b/pkg/schedule/schedulers/hot_region.go @@ -127,13 +127,13 @@ type baseHotScheduler struct { updateWriteTime time.Time } -func newBaseHotScheduler(opController *operator.Controller) *baseHotScheduler { +func newBaseHotScheduler(opController *operator.Controller, sampleDuration time.Duration, sampleInterval time.Duration) *baseHotScheduler { base := NewBaseScheduler(opController) ret := &baseHotScheduler{ BaseScheduler: base, types: []utils.RWType{utils.Write, utils.Read}, regionPendings: make(map[uint64]*pendingInfluence), - stHistoryLoads: statistics.NewStoreHistoryLoads(utils.DimLen), + stHistoryLoads: statistics.NewStoreHistoryLoads(utils.DimLen, sampleDuration, sampleInterval), r: rand.New(rand.NewSource(time.Now().UnixNano())), } for ty := resourceType(0); ty < resourceTypeLen; ty++ { @@ -180,6 +180,10 @@ func (h *baseHotScheduler) prepareForBalance(rw utils.RWType, cluster sche.Sched } } +func (h *baseHotScheduler) updateHistoryLoadConfig(sampleDuration, sampleInterval time.Duration) { + h.stHistoryLoads = h.stHistoryLoads.UpdateConfig(sampleDuration, sampleInterval) +} + // summaryPendingInfluence calculate the summary of pending Influence for each store // and clean the region from regionInfluence if they have ended operator. // It makes each dim rate or count become `weight` times to the origin value. @@ -233,7 +237,8 @@ type hotScheduler struct { } func newHotScheduler(opController *operator.Controller, conf *hotRegionSchedulerConfig) *hotScheduler { - base := newBaseHotScheduler(opController) + base := newBaseHotScheduler(opController, + conf.GetHistorySampleDuration(), conf.GetHistorySampleInterval()) ret := &hotScheduler{ name: HotRegionName, baseHotScheduler: base, @@ -292,6 +297,8 @@ func (h *hotScheduler) ReloadConfig() error { h.conf.RankFormulaVersion = newCfg.RankFormulaVersion h.conf.ForbidRWType = newCfg.ForbidRWType h.conf.SplitThresholds = newCfg.SplitThresholds + h.conf.HistorySampleDuration = newCfg.HistorySampleDuration + h.conf.HistorySampleInterval = newCfg.HistorySampleInterval return nil } @@ -324,6 +331,7 @@ func (h *hotScheduler) Schedule(cluster sche.SchedulerCluster, dryRun bool) ([]* func (h *hotScheduler) dispatch(typ utils.RWType, cluster sche.SchedulerCluster) []*operator.Operator { h.Lock() defer h.Unlock() + h.updateHistoryLoadConfig(h.conf.GetHistorySampleDuration(), h.conf.GetHistorySampleInterval()) h.prepareForBalance(typ, cluster) // it can not move earlier to support to use api and metrics. if h.conf.IsForbidRWType(typ) { diff --git a/pkg/schedule/schedulers/hot_region_config.go b/pkg/schedule/schedulers/hot_region_config.go index adf6480bacf..b336438830b 100644 --- a/pkg/schedule/schedulers/hot_region_config.go +++ b/pkg/schedule/schedulers/hot_region_config.go @@ -26,10 +26,12 @@ import ( "github.com/tikv/pd/pkg/errs" sche "github.com/tikv/pd/pkg/schedule/core" "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/statistics" "github.com/tikv/pd/pkg/statistics/utils" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/reflectutil" "github.com/tikv/pd/pkg/utils/syncutil" + "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/pkg/versioninfo" "github.com/unrolled/render" "go.uber.org/zap" @@ -76,6 +78,8 @@ func initHotRegionScheduleConfig() *hotRegionSchedulerConfig { RankFormulaVersion: "v2", ForbidRWType: "none", SplitThresholds: 0.2, + HistorySampleDuration: typeutil.NewDuration(statistics.DefaultHistorySampleDuration), + HistorySampleInterval: typeutil.NewDuration(statistics.DefaultHistorySampleInterval), } cfg.applyPrioritiesConfig(defaultPrioritiesConfig) return cfg @@ -104,6 +108,8 @@ func (conf *hotRegionSchedulerConfig) getValidConf() *hotRegionSchedulerConfig { RankFormulaVersion: conf.getRankFormulaVersionLocked(), ForbidRWType: conf.getForbidRWTypeLocked(), SplitThresholds: conf.SplitThresholds, + HistorySampleDuration: conf.HistorySampleDuration, + HistorySampleInterval: conf.HistorySampleInterval, } } @@ -147,6 +153,9 @@ type hotRegionSchedulerConfig struct { ForbidRWType string `json:"forbid-rw-type,omitempty"` // SplitThresholds is the threshold to split hot region if the first priority flow of on hot region exceeds it. SplitThresholds float64 `json:"split-thresholds"` + + HistorySampleDuration typeutil.Duration `json:"history-sample-duration"` + HistorySampleInterval typeutil.Duration `json:"history-sample-interval"` } func (conf *hotRegionSchedulerConfig) EncodeConfig() ([]byte, error) { @@ -305,6 +314,30 @@ func (conf *hotRegionSchedulerConfig) GetRankFormulaVersion() string { return conf.getRankFormulaVersionLocked() } +func (conf *hotRegionSchedulerConfig) GetHistorySampleDuration() time.Duration { + conf.RLock() + defer conf.RUnlock() + return conf.HistorySampleDuration.Duration +} + +func (conf *hotRegionSchedulerConfig) GetHistorySampleInterval() time.Duration { + conf.RLock() + defer conf.RUnlock() + return conf.HistorySampleInterval.Duration +} + +func (conf *hotRegionSchedulerConfig) SetHistorySampleDuration(d time.Duration) { + conf.Lock() + defer conf.Unlock() + conf.HistorySampleDuration = typeutil.NewDuration(d) +} + +func (conf *hotRegionSchedulerConfig) SetHistorySampleInterval(d time.Duration) { + conf.Lock() + defer conf.Unlock() + conf.HistorySampleInterval = typeutil.NewDuration(d) +} + func (conf *hotRegionSchedulerConfig) getRankFormulaVersionLocked() string { switch conf.RankFormulaVersion { case "v2": diff --git a/pkg/schedule/schedulers/hot_region_test.go b/pkg/schedule/schedulers/hot_region_test.go index 4b627b9a610..8b1893887db 100644 --- a/pkg/schedule/schedulers/hot_region_test.go +++ b/pkg/schedule/schedulers/hot_region_test.go @@ -40,6 +40,8 @@ import ( ) func init() { + // TODO: remove this global variable in the future. + // And use a function to create hot schduler for test. schedulePeerPr = 1.0 RegisterScheduler(utils.Write.String(), func(opController *operator.Controller, storage endpoint.ConfigStorage, decoder ConfigDecoder, removeSchedulerCb ...func(string) error) (Scheduler, error) { cfg := initHotRegionScheduleConfig() @@ -199,7 +201,6 @@ func newTestRegion(id uint64) *core.RegionInfo { func TestHotWriteRegionScheduleByteRateOnly(t *testing.T) { re := require.New(t) statistics.Denoising = false - statistics.HistorySampleDuration = 0 statisticsInterval = 0 checkHotWriteRegionScheduleByteRateOnly(re, false /* disable placement rules */) checkHotWriteRegionScheduleByteRateOnly(re, true /* enable placement rules */) @@ -404,6 +405,7 @@ func checkHotWriteRegionPlacement(re *require.Assertions, enablePlacementRules b tc.SetMaxReplicasWithLabel(enablePlacementRules, 3, labels...) hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.AddLabelsStore(1, 2, map[string]string{"zone": "z1", "host": "h1"}) @@ -457,6 +459,7 @@ func checkHotWriteRegionScheduleByteRateOnly(re *require.Assertions, enablePlace tc.SetMaxReplicasWithLabel(enablePlacementRules, 3, labels...) hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) // Add stores 1, 2, 3, 4, 5, 6 with region counts 3, 2, 2, 2, 0, 0. @@ -661,6 +664,7 @@ func TestHotWriteRegionScheduleByteRateOnlyWithTiFlash(t *testing.T) { sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb := sche.(*hotScheduler) + hb.conf.SetHistorySampleDuration(0) // Add TiKV stores 1, 2, 3, 4, 5, 6, 7 (Down) with region counts 3, 3, 2, 2, 0, 0, 0. // Add TiFlash stores 8, 9, 10, 11 with region counts 3, 1, 1, 0. @@ -852,6 +856,7 @@ func TestHotWriteRegionScheduleWithQuery(t *testing.T) { hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.QueryPriority, utils.BytePriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -889,6 +894,7 @@ func TestHotWriteRegionScheduleWithKeyRate(t *testing.T) { hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) @@ -1027,6 +1033,7 @@ func TestHotWriteRegionScheduleWithLeader(t *testing.T) { defer cancel() hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) re.NoError(err) tc.SetHotRegionCacheHitsThreshold(0) @@ -1095,6 +1102,7 @@ func checkHotWriteRegionScheduleWithPendingInfluence(re *require.Assertions, dim re.NoError(err) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) old := pendingAmpFactor pendingAmpFactor = 0.0 defer func() { @@ -1182,6 +1190,7 @@ func TestHotWriteRegionScheduleWithRuleEnabled(t *testing.T) { hb, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) re.NoError(err) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) key, err := hex.DecodeString("") @@ -1265,6 +1274,7 @@ func TestHotReadRegionScheduleByteRateOnly(t *testing.T) { re.NoError(err) hb := scheduler.(*hotScheduler) hb.conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} + hb.conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) // Add stores 1, 2, 3, 4, 5 with region counts 3, 2, 2, 2, 0. @@ -1389,6 +1399,7 @@ func TestHotReadRegionScheduleWithQuery(t *testing.T) { hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.RankFormulaVersion = "v1" + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -1425,6 +1436,7 @@ func TestHotReadRegionScheduleWithKeyRate(t *testing.T) { hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -1489,6 +1501,7 @@ func checkHotReadRegionScheduleWithPendingInfluence(re *require.Assertions, dim hb.(*hotScheduler).conf.MinorDecRatio = 1 hb.(*hotScheduler).conf.DstToleranceRatio = 1 hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) old := pendingAmpFactor pendingAmpFactor = 0.0 defer func() { @@ -2066,7 +2079,6 @@ func checkSortResult(re *require.Assertions, regions []uint64, hotPeers []*stati func TestInfluenceByRWType(t *testing.T) { re := require.New(t) - statistics.HistorySampleDuration = 0 originValue := schedulePeerPr defer func() { schedulePeerPr = originValue @@ -2080,6 +2092,7 @@ func TestInfluenceByRWType(t *testing.T) { re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.AddRegionStore(1, 20) @@ -2203,6 +2216,7 @@ func TestHotScheduleWithPriority(t *testing.T) { re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1.05) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1.05) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) // skip stddev check origin := stddevThreshold stddevThreshold = -1.0 @@ -2251,6 +2265,7 @@ func TestHotScheduleWithPriority(t *testing.T) { addRegionInfo(tc, utils.Read, []testRegionInfo{ {1, []uint64{1, 2, 3}, 2 * units.MiB, 2 * units.MiB, 0}, }) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} ops, _ = hb.Schedule(tc, false) re.Len(ops, 1) @@ -2264,6 +2279,7 @@ func TestHotScheduleWithPriority(t *testing.T) { hb, err = CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil) hb.(*hotScheduler).conf.WriteLeaderPriorities = []string{utils.KeyPriority, utils.BytePriority} hb.(*hotScheduler).conf.RankFormulaVersion = "v1" + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) re.NoError(err) // assert loose store picking @@ -2318,6 +2334,7 @@ func TestHotScheduleWithStddev(t *testing.T) { tc.AddRegionStore(4, 20) tc.AddRegionStore(5, 20) hb.(*hotScheduler).conf.StrictPickingStore = false + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) // skip uniform cluster tc.UpdateStorageWrittenStats(1, 5*units.MiB*utils.StoreHeartBeatReportInterval, 5*units.MiB*utils.StoreHeartBeatReportInterval) @@ -2369,6 +2386,7 @@ func TestHotWriteLeaderScheduleWithPriority(t *testing.T) { re.NoError(err) hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetSrcToleranceRatio(1) + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.SetHotRegionCacheHitsThreshold(0) diff --git a/pkg/schedule/schedulers/hot_region_v2_test.go b/pkg/schedule/schedulers/hot_region_v2_test.go index f5e21e02981..78a30cebaca 100644 --- a/pkg/schedule/schedulers/hot_region_v2_test.go +++ b/pkg/schedule/schedulers/hot_region_v2_test.go @@ -35,13 +35,13 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { defer cancel() statistics.Denoising = false statisticsInterval = 0 - statistics.HistorySampleDuration = 0 sche, err := CreateScheduler(utils.Write.String(), oc, storage.NewStorageWithMemoryBackend(), nil, nil) re.NoError(err) hb := sche.(*hotScheduler) hb.conf.SetDstToleranceRatio(0.0) hb.conf.SetSrcToleranceRatio(0.0) hb.conf.SetRankFormulaVersion("v1") + hb.conf.SetHistorySampleDuration(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -104,6 +104,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirst(t *testing.T) { hb.conf.SetDstToleranceRatio(0.0) hb.conf.SetSrcToleranceRatio(0.0) hb.conf.SetRankFormulaVersion("v1") + hb.conf.SetHistorySampleDuration(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -149,7 +150,6 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirstOnly(t *testing.T) { re := require.New(t) statistics.Denoising = false statisticsInterval = 0 - statistics.HistorySampleDuration = 0 cancel, _, tc, oc := prepareSchedulersTest() defer cancel() @@ -159,6 +159,7 @@ func TestHotWriteRegionScheduleWithRevertRegionsDimFirstOnly(t *testing.T) { hb.conf.SetDstToleranceRatio(0.0) hb.conf.SetSrcToleranceRatio(0.0) hb.conf.SetRankFormulaVersion("v1") + hb.conf.SetHistorySampleDuration(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -213,7 +214,6 @@ func TestHotReadRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { re := require.New(t) statistics.Denoising = false statisticsInterval = 0 - statistics.HistorySampleDuration = 0 cancel, _, tc, oc := prepareSchedulersTest() defer cancel() @@ -223,6 +223,7 @@ func TestHotReadRegionScheduleWithRevertRegionsDimSecond(t *testing.T) { hb.conf.SetDstToleranceRatio(0.0) hb.conf.SetSrcToleranceRatio(0.0) hb.conf.SetRankFormulaVersion("v1") + hb.conf.SetHistorySampleDuration(0) tc.SetClusterVersion(versioninfo.MinSupportedVersion(versioninfo.Version4_0)) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) @@ -276,7 +277,6 @@ func TestSkipUniformStore(t *testing.T) { re := require.New(t) statistics.Denoising = false statisticsInterval = 0 - statistics.HistorySampleDuration = 0 cancel, _, tc, oc := prepareSchedulersTest() defer cancel() @@ -286,6 +286,7 @@ func TestSkipUniformStore(t *testing.T) { hb.(*hotScheduler).conf.SetDstToleranceRatio(1) hb.(*hotScheduler).conf.SetRankFormulaVersion("v2") hb.(*hotScheduler).conf.ReadPriorities = []string{utils.BytePriority, utils.KeyPriority} + hb.(*hotScheduler).conf.SetHistorySampleDuration(0) tc.SetHotRegionCacheHitsThreshold(0) tc.AddRegionStore(1, 20) tc.AddRegionStore(2, 20) diff --git a/pkg/schedule/schedulers/shuffle_hot_region.go b/pkg/schedule/schedulers/shuffle_hot_region.go index 541be959fcf..a1448fbd041 100644 --- a/pkg/schedule/schedulers/shuffle_hot_region.go +++ b/pkg/schedule/schedulers/shuffle_hot_region.go @@ -95,7 +95,8 @@ type shuffleHotRegionScheduler struct { // newShuffleHotRegionScheduler creates an admin scheduler that random balance hot regions func newShuffleHotRegionScheduler(opController *operator.Controller, conf *shuffleHotRegionSchedulerConfig) Scheduler { - base := newBaseHotScheduler(opController) + base := newBaseHotScheduler(opController, + statistics.DefaultHistorySampleDuration, statistics.DefaultHistorySampleInterval) handler := newShuffleHotRegionHandler(conf) ret := &shuffleHotRegionScheduler{ baseHotScheduler: base, diff --git a/pkg/statistics/store_collection_test.go b/pkg/statistics/store_collection_test.go index 054e55a9fda..02e6350ffa4 100644 --- a/pkg/statistics/store_collection_test.go +++ b/pkg/statistics/store_collection_test.go @@ -98,7 +98,7 @@ func TestSummaryStoreInfos(t *testing.T) { rw := utils.Read kind := constant.LeaderKind collector := newTikvCollector() - storeHistoryLoad := NewStoreHistoryLoads(utils.DimLen) + storeHistoryLoad := NewStoreHistoryLoads(utils.DimLen, DefaultHistorySampleDuration, DefaultHistorySampleInterval) storeInfos := make(map[uint64]*StoreSummaryInfo) storeLoads := make(map[uint64][]float64) for _, storeID := range []int{1, 3} { @@ -130,7 +130,7 @@ func TestSummaryStoreInfos(t *testing.T) { } // case 2: put many elements into history load - historySampleInterval = 0 + storeHistoryLoad.sampleDuration = 0 for i := 1; i < 10; i++ { details = summaryStoresLoadByEngine(storeInfos, storeLoads, storeHistoryLoad, nil, rw, kind, collector) expect := []float64{2, 4, 10} diff --git a/pkg/statistics/store_load.go b/pkg/statistics/store_load.go index 79417b65b7e..c468024e3d6 100644 --- a/pkg/statistics/store_load.go +++ b/pkg/statistics/store_load.go @@ -245,24 +245,29 @@ func MaxLoad(a, b *StoreLoad) *StoreLoad { } } -var ( - // historySampleInterval is the sampling interval for history load. - historySampleInterval = 30 * time.Second - // HistorySampleDuration is the duration for saving history load. - HistorySampleDuration = 5 * time.Minute - defaultSize = 10 +const ( + // DefaultHistorySampleInterval is the sampling interval for history load. + DefaultHistorySampleInterval = 30 * time.Second + // DefaultHistorySampleDuration is the duration for saving history load. + DefaultHistorySampleDuration = 5 * time.Minute ) // StoreHistoryLoads records the history load of a store. type StoreHistoryLoads struct { // loads[read/write][leader/follower]-->[store id]-->history load - loads [utils.RWTypeLen][constant.ResourceKindLen]map[uint64]*storeHistoryLoad - dim int + loads [utils.RWTypeLen][constant.ResourceKindLen]map[uint64]*storeHistoryLoad + dim int + sampleInterval time.Duration + sampleDuration time.Duration } // NewStoreHistoryLoads creates a StoreHistoryLoads. -func NewStoreHistoryLoads(dim int) *StoreHistoryLoads { - st := StoreHistoryLoads{dim: dim} +func NewStoreHistoryLoads(dim int, sampleDuration time.Duration, sampleInterval time.Duration) *StoreHistoryLoads { + st := StoreHistoryLoads{ + dim: dim, + sampleDuration: sampleDuration, + sampleInterval: sampleInterval, + } for i := utils.RWType(0); i < utils.RWTypeLen; i++ { for j := constant.ResourceKind(0); j < constant.ResourceKindLen; j++ { st.loads[i][j] = make(map[uint64]*storeHistoryLoad) @@ -272,20 +277,24 @@ func NewStoreHistoryLoads(dim int) *StoreHistoryLoads { } // Add adds the store load to the history. -func (s *StoreHistoryLoads) Add(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind, loads []float64) { +func (s *StoreHistoryLoads) Add(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind, pointLoad []float64) { load, ok := s.loads[rwTp][kind][storeID] if !ok { - size := defaultSize - if historySampleInterval != 0 { - size = int(HistorySampleDuration / historySampleInterval) + size := int(DefaultHistorySampleDuration / DefaultHistorySampleInterval) + if s.sampleInterval != 0 { + size = int(s.sampleDuration / s.sampleInterval) + } + if s.sampleDuration == 0 { + size = 0 } - load = newStoreHistoryLoad(size, s.dim) + load = newStoreHistoryLoad(size, s.dim, s.sampleInterval) s.loads[rwTp][kind][storeID] = load } - load.add(loads) + load.add(pointLoad) } // Get returns the store loads from the history, not one time point. +// In another word, the result is [dim][time]. func (s *StoreHistoryLoads) Get(storeID uint64, rwTp utils.RWType, kind constant.ResourceKind) [][]float64 { load, ok := s.loads[rwTp][kind][storeID] if !ok { @@ -294,36 +303,46 @@ func (s *StoreHistoryLoads) Get(storeID uint64, rwTp utils.RWType, kind constant return load.get() } +// UpdateConfig updates the sample duration and interval. +func (s *StoreHistoryLoads) UpdateConfig(sampleDuration time.Duration, sampleInterval time.Duration) *StoreHistoryLoads { + if s.sampleDuration == sampleDuration && s.sampleInterval == sampleInterval { + return s + } + return NewStoreHistoryLoads(s.dim, sampleDuration, sampleInterval) +} + type storeHistoryLoad struct { update time.Time // loads is a circular buffer. // [dim] --> [1,2,3...] - loads [][]float64 - size int - count int + loads [][]float64 + size int + count int + sampleInterval time.Duration } -func newStoreHistoryLoad(size int, dim int) *storeHistoryLoad { +func newStoreHistoryLoad(size int, dimLen int, sampleInterval time.Duration) *storeHistoryLoad { return &storeHistoryLoad{ - loads: make([][]float64, dim), - size: size, + loads: make([][]float64, dimLen), + size: size, + sampleInterval: sampleInterval, } } // add adds the store load to the history. // eg. add([1,2,3]) --> [][]float64{{1}, {2}, {3}} -func (s *storeHistoryLoad) add(loads []float64) { +func (s *storeHistoryLoad) add(pointLoad []float64) { // reject if the loads length is not equal to the dimension. - if time.Since(s.update) < historySampleInterval || s.size == 0 || len(loads) != len(s.loads) { + if time.Since(s.update) < s.sampleInterval || s.size == 0 || len(pointLoad) != len(s.loads) { return } if s.count == 0 { - for i := range s.loads { - s.loads[i] = make([]float64, s.size) + for dim := range s.loads { + s.loads[dim] = make([]float64, s.size) } } - for i, v := range loads { - s.loads[i][s.count%s.size] = v + for dim, v := range pointLoad { + s.loads[dim][s.count%s.size] = v } s.count++ s.update = time.Now() diff --git a/pkg/statistics/store_load_test.go b/pkg/statistics/store_load_test.go index 67f2dff9cf9..67c9e53482f 100644 --- a/pkg/statistics/store_load_test.go +++ b/pkg/statistics/store_load_test.go @@ -16,6 +16,7 @@ package statistics import ( "testing" + "time" "github.com/stretchr/testify/require" "github.com/tikv/pd/pkg/core/constant" @@ -24,8 +25,7 @@ import ( func TestHistoryLoads(t *testing.T) { re := require.New(t) - historySampleInterval = 0 - historyLoads := NewStoreHistoryLoads(utils.DimLen) + historyLoads := NewStoreHistoryLoads(utils.DimLen, DefaultHistorySampleDuration, 0) loads := []float64{1.0, 2.0, 3.0} rwTp := utils.Read kind := constant.LeaderKind @@ -43,4 +43,20 @@ func TestHistoryLoads(t *testing.T) { expectLoads[utils.QueryDim][i] = 3.0 } re.EqualValues(expectLoads, historyLoads.Get(1, rwTp, kind)) + + historyLoads = NewStoreHistoryLoads(utils.DimLen, time.Millisecond, time.Millisecond) + historyLoads.Add(1, rwTp, kind, loads) + re.Len(historyLoads.Get(1, rwTp, kind)[0], 1) + + historyLoads = NewStoreHistoryLoads(utils.DimLen, time.Millisecond, time.Second) + historyLoads.Add(1, rwTp, kind, loads) + re.Empty(historyLoads.Get(1, rwTp, kind)[0]) + + historyLoads = NewStoreHistoryLoads(utils.DimLen, 0, time.Second) + historyLoads.Add(1, rwTp, kind, loads) + re.Empty(historyLoads.Get(1, rwTp, kind)[0]) + + historyLoads = NewStoreHistoryLoads(utils.DimLen, 0, 0) + historyLoads.Add(1, rwTp, kind, loads) + re.Empty(historyLoads.Get(1, rwTp, kind)[0]) } diff --git a/tests/server/api/scheduler_test.go b/tests/server/api/scheduler_test.go index 569741d026c..3b98be4bbb5 100644 --- a/tests/server/api/scheduler_test.go +++ b/tests/server/api/scheduler_test.go @@ -238,6 +238,8 @@ func (suite *scheduleTestSuite) checkAPI(cluster *tests.TestCluster) { "write-peer-priorities": []any{"byte", "key"}, "enable-for-tiflash": "true", "strict-picking-store": "true", + "history-sample-duration": "5m0s", + "history-sample-interval": "30s", } re.Equal(len(expectMap), len(resp), "expect %v, got %v", expectMap, resp) for key := range expectMap { diff --git a/tools/pd-ctl/tests/scheduler/scheduler_test.go b/tools/pd-ctl/tests/scheduler/scheduler_test.go index 18cfc91f2a4..b5a2128752b 100644 --- a/tools/pd-ctl/tests/scheduler/scheduler_test.go +++ b/tools/pd-ctl/tests/scheduler/scheduler_test.go @@ -406,6 +406,8 @@ func (suite *schedulerTestSuite) checkScheduler(cluster *pdTests.TestCluster) { "enable-for-tiflash": "true", "rank-formula-version": "v2", "split-thresholds": 0.2, + "history-sample-duration": "5m0s", + "history-sample-interval": "30s", } checkHotSchedulerConfig := func(expect map[string]any) { testutil.Eventually(re, func() bool { @@ -476,6 +478,26 @@ func (suite *schedulerTestSuite) checkScheduler(cluster *pdTests.TestCluster) { re.Contains(echo, "Success!") checkHotSchedulerConfig(expected1) + expected1["history-sample-duration"] = "1m0s" + echo = mustExec(re, cmd, []string{"-u", pdAddr, "scheduler", "config", "balance-hot-region-scheduler", "set", "history-sample-duration", "1m"}, nil) + re.Contains(echo, "Success!") + checkHotSchedulerConfig(expected1) + + expected1["history-sample-interval"] = "1s" + echo = mustExec(re, cmd, []string{"-u", pdAddr, "scheduler", "config", "balance-hot-region-scheduler", "set", "history-sample-interval", "1s"}, nil) + re.Contains(echo, "Success!") + checkHotSchedulerConfig(expected1) + + expected1["history-sample-duration"] = "0s" + echo = mustExec(re, cmd, []string{"-u", pdAddr, "scheduler", "config", "balance-hot-region-scheduler", "set", "history-sample-duration", "0s"}, nil) + re.Contains(echo, "Success!") + checkHotSchedulerConfig(expected1) + + expected1["history-sample-interval"] = "0s" + echo = mustExec(re, cmd, []string{"-u", pdAddr, "scheduler", "config", "balance-hot-region-scheduler", "set", "history-sample-interval", "0s"}, nil) + re.Contains(echo, "Success!") + checkHotSchedulerConfig(expected1) + // test compatibility re.Equal("2.0.0", leaderServer.GetClusterVersion().String()) for _, store := range stores {