From 076a2af3d6f0f528c10a1e1ee362d88ac08a23a6 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 2 Nov 2022 09:26:17 +0800 Subject: [PATCH 01/16] *: support mysql compatible auto_increment, the client side changes --- autoid_service/autoid.go | 32 ++++++ ddl/schematracker/checker.go | 22 +++- go.mod | 2 + go.sum | 10 +- infoschema/builder.go | 3 +- meta/autoid/autoid.go | 51 +++++++++ meta/autoid/autoid_service.go | 207 ++++++++++++++++++++++++++++++++++ metrics/metrics.go | 1 + metrics/session.go | 9 ++ server/server_test.go | 16 +-- server/tidb_test.go | 2 +- 11 files changed, 336 insertions(+), 19 deletions(-) create mode 100644 autoid_service/autoid.go create mode 100644 meta/autoid/autoid_service.go diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go new file mode 100644 index 0000000000000..eb23a0edefdc7 --- /dev/null +++ b/autoid_service/autoid.go @@ -0,0 +1,32 @@ +package autoid + +import ( + "context" + + "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/kv" + "google.golang.org/grpc" +) + +type mockClient struct { +} + +func (m *mockClient) AllocAutoID(ctx context.Context, in *autoid.AutoIDRequest, opts ...grpc.CallOption) (*autoid.AutoIDResponse, error) { + panic("TODO") +} + +func (m *mockClient) Rebase(ctx context.Context, in *autoid.RebaseRequest, opts ...grpc.CallOption) (*autoid.RebaseResponse, error) { + panic("TODO") +} + +var global = make(map[string]*mockClient) + +func MockForTest(store kv.Storage) *mockClient { + uuid := store.UUID() + ret, ok := global[uuid] + if !ok { + ret = &mockClient{} + global[uuid] = ret + } + return ret +} diff --git a/ddl/schematracker/checker.go b/ddl/schematracker/checker.go index 3a09db2f19c36..018c93f6f416e 100644 --- a/ddl/schematracker/checker.go +++ b/ddl/schematracker/checker.go @@ -17,6 +17,7 @@ package schematracker import ( "bytes" "context" + "crypto/tls" "fmt" "strings" "time" @@ -553,15 +554,34 @@ func (d Checker) MoveJobFromTable2Queue() error { // StorageDDLInjector wraps kv.Storage to inject checker to domain's DDL in bootstrap time. type StorageDDLInjector struct { kv.Storage + kv.EtcdBackend Injector func(ddl.DDL) *Checker } +var _ kv.EtcdBackend = StorageDDLInjector{} + +func (s StorageDDLInjector) EtcdAddrs() ([]string, error) { + return s.EtcdBackend.EtcdAddrs() +} + +func (s StorageDDLInjector) TLSConfig() *tls.Config { + return s.EtcdBackend.TLSConfig() +} + +func (s StorageDDLInjector) StartGCWorker() error { + return s.EtcdBackend.StartGCWorker() +} + // NewStorageDDLInjector creates a new StorageDDLInjector to inject Checker. func NewStorageDDLInjector(s kv.Storage) kv.Storage { - return StorageDDLInjector{ + ret := StorageDDLInjector{ Storage: s, Injector: NewChecker, } + if ebd, ok := s.(kv.EtcdBackend); ok { + ret.EtcdBackend = ebd + } + return ret } // UnwrapStorage unwraps StorageDDLInjector for one level. diff --git a/go.mod b/go.mod index 76ee052056499..32ba9ed3ae0b4 100644 --- a/go.mod +++ b/go.mod @@ -251,3 +251,5 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) + +replace github.com/pingcap/kvproto => github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 diff --git a/go.sum b/go.sum index 1b468866f3891..9920da6bd84ce 100644 --- a/go.sum +++ b/go.sum @@ -338,7 +338,6 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -367,7 +366,6 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -774,9 +772,6 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172 h1:FYgKV9znRQmzVrrJDZ0gUfMIvKLAMU1tu1UKJib8bEQ= -github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -928,6 +923,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= +github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 h1:+sVrTp/1hSRuQM+dLDl4OnjqPANDXuNFZu+EhheAWuw= +github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c h1:NvQHWk0GeXSLEBbmGMPnDMc0to0a3ogzgIRbTKw8MHI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= @@ -1131,7 +1128,6 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1487,7 +1483,6 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1568,7 +1563,6 @@ google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljW google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= -google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= diff --git a/infoschema/builder.go b/infoschema/builder.go index 8bc907516c0d3..8ff50a8b50435 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -699,7 +699,8 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i tblVer := autoid.AllocOptionTableInfoVersion(tblInfo.Version) switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, tblVer) + idCacheOpt := autoid.CustomAutoIncCacheOption(tblInfo.AutoIdCache) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, tblVer, idCacheOpt) allocs = append(allocs, newAlloc) case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index e20c0ed898f13..ebb43256faa09 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -26,6 +26,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + autoid "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/tidb/util/mathutil" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" tikvutil "github.com/tikv/client-go/v2/util" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -466,6 +468,7 @@ func (alloc *allocator) ForceRebase(requiredBase int64) error { if requiredBase == -1 { return ErrAutoincReadFailed.GenWithStack("Cannot force rebase the next global ID to '0'") } + alloc.mu.Lock() defer alloc.mu.Unlock() startTime := time.Now() @@ -533,6 +536,45 @@ func NextStep(curStep int64, consumeDur time.Duration) int64 { return res } +func newSinglePointAlloc(store kv.Storage, dbID, tblID int64, isUnsigned bool) *singlePointAlloc { + ebd, ok := store.(kv.EtcdBackend) + if !ok { + // newSinglePointAlloc fail because not etcd background + // This could happen in the server package unit test + return nil + } + + addrs, err := ebd.EtcdAddrs() + if err != nil { + panic(err) + } + spa := &singlePointAlloc{ + dbID: dbID, + tblID: tblID, + isUnsigned: isUnsigned, + } + if len(addrs) > 0 { + etcdCli, err := clientv3.New(clientv3.Config{ + Endpoints: addrs, + TLS: ebd.TLSConfig(), + }) + if err != nil { + panic(err) + } + spa.clientDiscover = clientDiscover{etcdCli: etcdCli} + } else { + spa.clientDiscover = clientDiscover{} + spa.mu.AutoIDAllocClient = autoid.MockForTest(store) + } + + failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { + if val.(bool) { + spa = nil + } + }) + return spa +} + // NewAllocator returns a new auto increment id generator on the store. func NewAllocator(store kv.Storage, dbID, tbID int64, isUnsigned bool, allocType AllocatorType, opts ...AllocOption) Allocator { @@ -548,6 +590,15 @@ func NewAllocator(store kv.Storage, dbID, tbID int64, isUnsigned bool, for _, fn := range opts { fn.ApplyOn(alloc) } + + // Use the MySQL compatible AUTO_INCREMENT mode. + if allocType == RowIDAllocType && alloc.customStep && alloc.step == 1 { + alloc1 := newSinglePointAlloc(store, dbID, tbID, isUnsigned) + if alloc1 != nil { + return alloc1 + } + } + return alloc } diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go new file mode 100644 index 0000000000000..35eabf0510dd7 --- /dev/null +++ b/meta/autoid/autoid_service.go @@ -0,0 +1,207 @@ +// Copyright 2022 PingCAP, Inc. +// +// 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 autoid + +import ( + "context" + "strings" + "sync" + "time" + + "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/metrics" + clientv3 "go.etcd.io/etcd/client/v3" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +var _ Allocator = &singlePointAlloc{} + +type singlePointAlloc struct { + dbID int64 + tblID int64 + lastAllocated int64 + isUnsigned bool + clientDiscover +} + +type clientDiscover struct { + // This the etcd client for service discover + etcdCli *clientv3.Client + // This is the real client for the AutoIDAlloc service + mu struct { + sync.RWMutex + autoid.AutoIDAllocClient + } +} + +const ( + autoIDLeaderPath = "tidb/autoid/leader" +) + +func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClient, error) { + d.mu.RLock() + cli := d.mu.AutoIDAllocClient + if cli != nil { + d.mu.RUnlock() + return cli, nil + } + d.mu.RUnlock() + + resp, err := d.etcdCli.Get(ctx, autoIDLeaderPath, clientv3.WithFirstCreate()...) + if err != nil { + return nil, errors.Trace(err) + } + + if len(resp.Kvs) == 0 { + return nil, errors.New("autoid service leader not found") + } + + addr := string(resp.Kvs[0].Value) + grpcConn, err := grpc.Dial(addr, grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return nil, errors.Trace(err) + } + cli = autoid.NewAutoIDAllocClient(grpcConn) + d.mu.Lock() + d.mu.AutoIDAllocClient = cli + d.mu.Unlock() + return cli, nil +} + +// Alloc allocs N consecutive autoID for table with tableID, returning (min, max] of the allocated autoID batch. +// The consecutive feature is used to insert multiple rows in a statement. +// increment & offset is used to validate the start position (the allocator's base is not always the last allocated id). +// The returned range is (min, max]: +// case increment=1 & offset=1: you can derive the ids like min+1, min+2... max. +// case increment=x & offset=y: you firstly need to seek to firstID by `SeekToFirstAutoIDXXX`, then derive the IDs like firstID, firstID + increment * 2... in the caller. +func (sp *singlePointAlloc) Alloc(ctx context.Context, n uint64, increment, offset int64) (int64, int64, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("autoid.Alloc", opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + + if !validIncrementAndOffset(increment, offset) { + return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) + } + +retry: + cli, err := sp.GetClient(ctx) + if err != nil { + return 0, 0, errors.Trace(err) + } + + start := time.Now() + resp, err := cli.AllocAutoID(ctx, &autoid.AutoIDRequest{ + DbID: sp.dbID, + TblID: sp.tblID, + N: n, + Increment: increment, + Offset: offset, + IsUnsigned: sp.isUnsigned, + }) + if err != nil { + if strings.Contains(err.Error(), "rpc error") { + sp.mu.Lock() + sp.mu.AutoIDAllocClient = nil + sp.mu.Unlock() + goto retry + } + return 0, 0, errors.Trace(err) + } + + du := time.Since(start) + metrics.AutoIDReqDuration.Observe(du.Seconds()) + if err == nil { + sp.lastAllocated = resp.Min + } + return resp.Min, resp.Max, err +} + +// AllocSeqCache allocs sequence batch value cached in table level(rather than in alloc), the returned range covering +// the size of sequence cache with it's increment. The returned round indicates the sequence cycle times if it is with +// cycle option. +func (sp *singlePointAlloc) AllocSeqCache() (min int64, max int64, round int64, err error) { + return 0, 0, 0, errors.New("AllocSeqCache not implemented") +} + +// Rebase rebases the autoID base for table with tableID and the new base value. +// If allocIDs is true, it will allocate some IDs and save to the cache. +// If allocIDs is false, it will not allocate IDs. +func (sp *singlePointAlloc) Rebase(ctx context.Context, newBase int64, allocIDs bool) error { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("autoid.Rebase", opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + + return sp.rebase(ctx, newBase, allocIDs, false) +} + +func (sp *singlePointAlloc) rebase(ctx context.Context, newBase int64, allocIDs bool, force bool) error { + cli, err := sp.GetClient(ctx) + if err != nil { + return errors.Trace(err) + } + _, err = cli.Rebase(ctx, &autoid.RebaseRequest{ + DbID: sp.dbID, + TblID: sp.tblID, + Base: newBase, + Force: force, + IsUnsigned: sp.isUnsigned, + }) + if err == nil { + sp.lastAllocated = newBase + } + return err +} + +// ForceRebase set the next global auto ID to newBase. +func (sp *singlePointAlloc) ForceRebase(newBase int64) error { + if newBase == -1 { + return ErrAutoincReadFailed.GenWithStack("Cannot force rebase the next global ID to '0'") + } + return sp.rebase(context.Background(), newBase, false, true) +} + +// RebaseSeq rebases the sequence value in number axis with tableID and the new base value. +func (sp *singlePointAlloc) RebaseSeq(newBase int64) (int64, bool, error) { + return 0, false, errors.New("RebaseSeq not implemented") +} + +// Base return the current base of Allocator. +func (sp *singlePointAlloc) Base() int64 { + return sp.lastAllocated +} + +// End is only used for test. +func (sp *singlePointAlloc) End() int64 { + return sp.lastAllocated +} + +// NextGlobalAutoID returns the next global autoID. +// Used by 'show create table', 'alter table auto_increment = xxx' +func (sp *singlePointAlloc) NextGlobalAutoID() (int64, error) { + // return sp.lastAllocated + 1, nil + _, max, err := sp.Alloc(context.Background(), 0, 1, 1) + return max + 1, err +} + +func (sp *singlePointAlloc) GetType() AllocatorType { + return RowIDAllocType +} diff --git a/metrics/metrics.go b/metrics/metrics.go index 3d6ba83979f45..762c31635257c 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -203,6 +203,7 @@ func RegisterMetrics() { prometheus.MustRegister(GetCheckpointBatchSize) prometheus.MustRegister(RegionCheckpointRequest) prometheus.MustRegister(RegionCheckpointFailure) + prometheus.MustRegister(AutoIDReqDuration) prometheus.MustRegister(RCCheckTSWriteConfilictCounter) tikvmetrics.InitMetrics(TiDB, TiKVClient) diff --git a/metrics/session.go b/metrics/session.go index a54ab8e4cb8d4..e6b069869ed2c 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -18,6 +18,15 @@ import "github.com/prometheus/client_golang/prometheus" // Session metrics. var ( + AutoIDReqDuration = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "meta", + Name: "autoid_duration_seconds", + Help: "Bucketed histogram of processing time (s) in parse SQL.", + Buckets: prometheus.ExponentialBuckets(0.00004, 2, 28), // 40us ~ 1.5h + }) + SessionExecuteParseDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", diff --git a/server/server_test.go b/server/server_test.go index 8639f98076232..623a4d3313628 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -626,8 +626,8 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, - "Warning 1062 Duplicate entry '1' for key 'idx'", - "Warning 1062 Duplicate entry '2' for key 'idx'") + "Warning 1062 Duplicate entry '1' for key 't.idx'", + "Warning 1062 Duplicate entry '2' for key 't.idx'") require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") @@ -680,8 +680,8 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, - "Warning 1062 Duplicate entry '1-2' for key 'idx'", - "Warning 1062 Duplicate entry '2-2' for key 'idx'") + "Warning 1062 Duplicate entry '1-2' for key 't.idx'", + "Warning 1062 Duplicate entry '2-2' for key 't.idx'") require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") @@ -735,8 +735,8 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, - "Warning 1062 Duplicate entry '1' for key 'idx'", - "Warning 1062 Duplicate entry '2' for key 'idx'") + "Warning 1062 Duplicate entry '1' for key 't.idx'", + "Warning 1062 Duplicate entry '2' for key 't.idx'") require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") @@ -789,7 +789,7 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T require.NoError(t, err) require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") - cli.checkRows(t, rows, "Warning 1062 Duplicate entry 'w-1' for key 'idx'") + cli.checkRows(t, rows, "Warning 1062 Duplicate entry 'w-1' for key 't.idx'") require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2", "e 5 5", "n 9 9") @@ -807,7 +807,7 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list", - "Warning 1062 Duplicate entry 'w-1' for key 'idx'") + "Warning 1062 Duplicate entry 'w-1' for key 't.idx'") require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2", "w 3 3", "e 5 5", "e 8 8", "n 9 9") diff --git a/server/tidb_test.go b/server/tidb_test.go index 633dab3cef8af..fc176613768ba 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1671,7 +1671,7 @@ func TestTopSQLCPUProfile(t *testing.T) { dbt.MustExec("alter table t drop index if exists idx_b") _, err := db.Exec(addIndexStr) require.NotNil(t, err) - require.Equal(t, "Error 1062: Duplicate entry '1' for key 'idx_b'", err.Error()) + require.Equal(t, "Error 1062: Duplicate entry '1' for key 't.idx_b'", err.Error()) } check = func() { checkFn(addIndexStr, "") From 206ed6a98659b5994b8cb1f4c029863f3360dc70 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 2 Nov 2022 09:38:21 +0800 Subject: [PATCH 02/16] *: support mysql compatible auto_increment, the server side changes --- autoid_service/autoid.go | 442 +++++++++++++++++++++++++++++++++++++++ go.mod | 2 + go.sum | 10 +- server/http_status.go | 26 +++ server/server.go | 6 + 5 files changed, 478 insertions(+), 8 deletions(-) create mode 100644 autoid_service/autoid.go diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go new file mode 100644 index 0000000000000..23d0555eb1497 --- /dev/null +++ b/autoid_service/autoid.go @@ -0,0 +1,442 @@ +package autoid + +import ( + "context" + "math" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/mathutil" + clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" + "google.golang.org/grpc" +) + +var ( + ErrAutoincReadFailed = errors.New("auto increment action failed") +) + +const ( + autoIDLeaderPath = "tidb/autoid/leader" +) + +type autoIDKey struct { + dbID int64 + tblID int64 +} + +type autoIDValue struct { + sync.Mutex + base int64 + end int64 + isUnsigned bool + token chan struct{} +} + +func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage, dbID, tblID int64, isUnsigned bool, + n uint64, increment, offset int64) (min int64, max int64, err error) { + // Check offset rebase if necessary. + if uint64(offset-1) > uint64(alloc.base) { + if err := alloc.rebase4Unsigned(ctx, store, dbID, tblID, uint64(offset-1)); err != nil { + return 0, 0, err + } + } + // calcNeededBatchSize calculates the total batch size needed. + n1 := calcNeededBatchSize(alloc.base, int64(n), increment, offset, isUnsigned) + + // The local rest is not enough for alloc, skip it. + if uint64(alloc.base)+uint64(n1) > uint64(alloc.end) || alloc.base == 0 { + var newBase, newEnd int64 + nextStep := int64(batch) + // Although it may skip a segment here, we still treat it as consumed. + + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + var err1 error + newBase, err1 = idAcc.Get() + if err1 != nil { + return err1 + } + // calcNeededBatchSize calculates the total batch size needed on new base. + n1 = calcNeededBatchSize(newBase, int64(n), increment, offset, isUnsigned) + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. + if nextStep < n1 { + nextStep = n1 + } + tmpStep := int64(mathutil.Min(math.MaxUint64-uint64(newBase), uint64(nextStep))) + // The global rest is not enough for alloc. + if tmpStep < n1 { + return ErrAutoincReadFailed + } + newEnd, err1 = idAcc.Inc(tmpStep) + return err1 + }) + if err != nil { + return 0, 0, err + } + if uint64(newBase) == math.MaxUint64 { + return 0, 0, ErrAutoincReadFailed + } + alloc.base, alloc.end = newBase, newEnd + } + min = alloc.base + // Use uint64 n directly. + alloc.base = int64(uint64(alloc.base) + uint64(n1)) + return min, alloc.base, nil +} + +func (alloc *autoIDValue) alloc4Signed(ctx context.Context, + store kv.Storage, + dbID, tblID int64, + isUnsigned bool, + n uint64, increment, offset int64) (min int64, max int64, err error) { + // Check offset rebase if necessary. + if offset-1 > alloc.base { + if err := alloc.rebase4Signed(ctx, store, dbID, tblID, offset-1); err != nil { + return 0, 0, err + } + } + // calcNeededBatchSize calculates the total batch size needed. + n1 := calcNeededBatchSize(alloc.base, int64(n), increment, offset, isUnsigned) + + // Condition alloc.base+N1 > alloc.end will overflow when alloc.base + N1 > MaxInt64. So need this. + if math.MaxInt64-alloc.base <= n1 { + return 0, 0, ErrAutoincReadFailed + } + + // The local rest is not enough for allocN, skip it. + // If alloc.base is 0, the alloc may not be initialized, force fetch from remote. + if alloc.base+n1 > alloc.end || alloc.base == 0 { + var newBase, newEnd int64 + nextStep := int64(batch) + + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + var err1 error + newBase, err1 = idAcc.Get() + if err1 != nil { + return err1 + } + // calcNeededBatchSize calculates the total batch size needed on global base. + n1 = calcNeededBatchSize(newBase, int64(n), increment, offset, isUnsigned) + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. + if nextStep < n1 { + nextStep = n1 + } + tmpStep := mathutil.Min(math.MaxInt64-newBase, nextStep) + // The global rest is not enough for alloc. + if tmpStep < n1 { + return ErrAutoincReadFailed + } + newEnd, err1 = idAcc.Inc(tmpStep) + return err1 + }) + if err != nil { + return 0, 0, err + } + if newBase == math.MaxInt64 { + return 0, 0, ErrAutoincReadFailed + } + alloc.base, alloc.end = newBase, newEnd + } + min = alloc.base + alloc.base += n1 + return min, alloc.base, nil +} + +func (alloc *autoIDValue) rebase4Unsigned(ctx context.Context, + store kv.Storage, + dbID, tblID int64, + requiredBase uint64) error { + // Satisfied by alloc.base, nothing to do. + if requiredBase <= uint64(alloc.base) { + return nil + } + // Satisfied by alloc.end, need to update alloc.base. + if requiredBase <= uint64(alloc.end) { + alloc.base = int64(requiredBase) + return nil + } + + var newBase, newEnd uint64 + startTime := time.Now() + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + currentEnd, err1 := idAcc.Get() + if err1 != nil { + return err1 + } + uCurrentEnd := uint64(currentEnd) + newBase = mathutil.Max(uCurrentEnd, requiredBase) + newEnd = mathutil.Min(math.MaxUint64-uint64(batch), newBase) + uint64(batch) + _, err1 = idAcc.Inc(int64(newEnd - uCurrentEnd)) + return err1 + }) + metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + if err != nil { + return err + } + alloc.base, alloc.end = int64(newBase), int64(newEnd) + return nil +} + +func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, dbID, tblID int64, requiredBase int64) error { + // Satisfied by alloc.base, nothing to do. + if requiredBase <= alloc.base { + return nil + } + // Satisfied by alloc.end, need to update alloc.base. + if requiredBase <= alloc.end { + alloc.base = requiredBase + return nil + } + + var newBase, newEnd int64 + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + currentEnd, err1 := idAcc.Get() + if err1 != nil { + return err1 + } + newBase = mathutil.Max(currentEnd, requiredBase) + newEnd = mathutil.Min(math.MaxInt64-batch, newBase) + batch + _, err1 = idAcc.Inc(newEnd - currentEnd) + return err1 + }) + if err != nil { + return err + } + alloc.base, alloc.end = newBase, newEnd + return nil +} + +type Service struct { + autoIDLock sync.Mutex + autoIDMap map[autoIDKey]*autoIDValue + + leaderShip owner.Manager + store kv.Storage +} + +func New(selfAddr string, etcdAddr []string, store kv.Storage) *Service { + cli, err := clientv3.New(clientv3.Config{ + Endpoints: etcdAddr, + DialTimeout: time.Second, + }) + if err != nil { + panic(err) + } + + l := owner.NewOwnerManager(context.Background(), cli, "autoid", selfAddr, autoIDLeaderPath) + err = l.CampaignOwner() + if err != nil { + panic(err) + } + + return &Service{ + autoIDMap: make(map[autoIDKey]*autoIDValue), + leaderShip: l, + store: store, + } +} + +type mockClient struct { + Service +} + +func (m *mockClient) AllocAutoID(ctx context.Context, in *autoid.AutoIDRequest, opts ...grpc.CallOption) (*autoid.AutoIDResponse, error) { + return m.Service.AllocAutoID(ctx, in) +} + +func (m *mockClient) Rebase(ctx context.Context, in *autoid.RebaseRequest, opts ...grpc.CallOption) (*autoid.RebaseResponse, error) { + return m.Service.Rebase(ctx, in) +} + +var global = make(map[string]*mockClient) + +func MockForTest(store kv.Storage) *mockClient { + uuid := store.UUID() + ret, ok := global[uuid] + if !ok { + ret = &mockClient{ + Service{ + autoIDMap: make(map[autoIDKey]*autoIDValue), + leaderShip: nil, + store: store, + }, + } + global[uuid] = ret + } + return ret +} + +func (s *Service) Close() { + if s.leaderShip != nil { + for k, v := range s.autoIDMap { + if v.base > 0 { + err := v.forceRebase(context.Background(), s.store, k.dbID, k.tblID, v.base, v.isUnsigned) + if err != nil { + logutil.BgLogger().Warn("[autoid service] save cached ID fail when service exit", + zap.Int64("db id", k.dbID), + zap.Int64("table id", k.tblID), + zap.Int64("value", v.base), + zap.Error(err)) + } + } + } + s.leaderShip.Cancel() + } +} + +// seekToFirstAutoIDSigned seeks to the next valid signed position. +func seekToFirstAutoIDSigned(base, increment, offset int64) int64 { + nr := (base + increment - offset) / increment + nr = nr*increment + offset + return nr +} + +// seekToFirstAutoIDUnSigned seeks to the next valid unsigned position. +func seekToFirstAutoIDUnSigned(base, increment, offset uint64) uint64 { + nr := (base + increment - offset) / increment + nr = nr*increment + offset + return nr +} + +func calcNeededBatchSize(base, n, increment, offset int64, isUnsigned bool) int64 { + if increment == 1 { + return n + } + if isUnsigned { + // SeekToFirstAutoIDUnSigned seeks to the next unsigned valid position. + nr := seekToFirstAutoIDUnSigned(uint64(base), uint64(increment), uint64(offset)) + // calculate the total batch size needed. + nr += (uint64(n) - 1) * uint64(increment) + return int64(nr - uint64(base)) + } + nr := seekToFirstAutoIDSigned(base, increment, offset) + // calculate the total batch size needed. + nr += (n - 1) * increment + return nr - base +} + +const batch = 4000 + +// AllocID implements gRPC PDServer. +func (s *Service) AllocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*autoid.AutoIDResponse, error) { + var res *autoid.AutoIDResponse + for { + var err error + res, err = s.allocAutoID(ctx, req) + if err != nil { + return nil, errors.Trace(err) + } + if res != nil { + break + } + } + return res, nil +} + +func (s *Service) getAlloc(dbID, tblID int64, isUnsigned bool) *autoIDValue { + key := autoIDKey{dbID: dbID, tblID: tblID} + s.autoIDLock.Lock() + defer s.autoIDLock.Unlock() + + val, ok := s.autoIDMap[key] + if !ok { + val = &autoIDValue{ + isUnsigned: isUnsigned, + token: make(chan struct{}, 1), + } + s.autoIDMap[key] = val + } + + return val +} + +func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*autoid.AutoIDResponse, error) { + if s.leaderShip != nil && !s.leaderShip.IsOwner() { + return nil, errors.New("not leader") + } + + val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned) + + if req.N == 0 && val.base != 0 { + base := val.base + return &autoid.AutoIDResponse{ + Min: base, + Max: base, + }, nil + } + + val.Lock() + defer val.Unlock() + + var min, max int64 + var err error + if req.IsUnsigned { + min, max, err = val.alloc4Unsigned(ctx, s.store, req.DbID, req.TblID, req.IsUnsigned, req.N, req.Increment, req.Offset) + } else { + min, max, err = val.alloc4Signed(ctx, s.store, req.DbID, req.TblID, req.IsUnsigned, req.N, req.Increment, req.Offset) + } + + return &autoid.AutoIDResponse{ + Min: min, + Max: max, + }, err +} + +func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbID, tblID, requiredBase int64, isUnsigned bool) error { + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + currentEnd, err1 := idAcc.Get() + if err1 != nil { + return err1 + } + var step int64 + if !isUnsigned { + step = requiredBase - currentEnd + } else { + uRequiredBase, uCurrentEnd := uint64(requiredBase), uint64(currentEnd) + step = int64(uRequiredBase - uCurrentEnd) + } + _, err1 = idAcc.Inc(step) + return err1 + }) + if err != nil { + return err + } + alloc.base, alloc.end = requiredBase, requiredBase + return nil +} + +func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoid.RebaseResponse, error) { + if s.leaderShip != nil && !s.leaderShip.IsOwner() { + return nil, errors.New("not leader") + } + + val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned) + if req.Force { + val.forceRebase(ctx, s.store, req.DbID, req.TblID, req.Base, req.IsUnsigned) + } + + var err error + if req.IsUnsigned { + err = val.rebase4Unsigned(ctx, s.store, req.DbID, req.TblID, uint64(req.Base)) + } else { + err = val.rebase4Signed(ctx, s.store, req.DbID, req.TblID, req.Base) + } + return &autoid.RebaseResponse{}, err +} diff --git a/go.mod b/go.mod index 76ee052056499..32ba9ed3ae0b4 100644 --- a/go.mod +++ b/go.mod @@ -251,3 +251,5 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) + +replace github.com/pingcap/kvproto => github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 diff --git a/go.sum b/go.sum index 1b468866f3891..9920da6bd84ce 100644 --- a/go.sum +++ b/go.sum @@ -338,7 +338,6 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -367,7 +366,6 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -774,9 +772,6 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172 h1:FYgKV9znRQmzVrrJDZ0gUfMIvKLAMU1tu1UKJib8bEQ= -github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -928,6 +923,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= +github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 h1:+sVrTp/1hSRuQM+dLDl4OnjqPANDXuNFZu+EhheAWuw= +github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c h1:NvQHWk0GeXSLEBbmGMPnDMc0to0a3ogzgIRbTKw8MHI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= @@ -1131,7 +1128,6 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1487,7 +1483,6 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1568,7 +1563,6 @@ google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljW google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= -google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= diff --git a/server/http_status.go b/server/http_status.go index 5c815b8ed1886..20c6a7ae550e0 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -38,10 +38,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/fn" + pb "github.com/pingcap/kvproto/pkg/autoid" + autoid "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/cpuprofile" "github.com/pingcap/tidb/util/logutil" @@ -457,6 +460,29 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) { statusServer := &http.Server{Addr: s.statusAddr, Handler: CorsHandler{handler: serverMux, cfg: s.cfg}} grpcServer := NewRPCServer(s.cfg, s.dom, s) service.RegisterChannelzServiceToServer(grpcServer) + if s.cfg.Store == "tikv" { + for { + fullPath := fmt.Sprintf("tikv://%s", s.cfg.Path) + store, err := store.New(fullPath) + if err != nil { + logutil.BgLogger().Error("new tikv store fail", zap.Error(err)) + break + } + ebd, ok := store.(kv.EtcdBackend) + if !ok { + break + } + etcdAddr, err := ebd.EtcdAddrs() + if err != nil { + logutil.BgLogger().Error("tikv store not etcd background", zap.Error(err)) + break + } + service := autoid.New(s.statusListener.Addr().String(), etcdAddr, store) + pb.RegisterAutoIDAllocServer(grpcServer, service) + s.autoIDService = service + break + } + } s.statusServer = statusServer s.grpcServer = grpcServer diff --git a/server/server.go b/server/server.go index 5efef687ecdb0..429c362c3da2c 100644 --- a/server/server.go +++ b/server/server.go @@ -37,6 +37,7 @@ import ( "math/rand" "net" "net/http" //nolint:goimports + // For pprof _ "net/http/pprof" // #nosec G108 "os" @@ -48,6 +49,7 @@ import ( "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" + autoid "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" @@ -140,6 +142,7 @@ type Server struct { sessionMapMutex sync.Mutex internalSessions map[interface{}]struct{} + autoIDService *autoid.Service } // ConnectionCount gets current connection count. @@ -498,6 +501,9 @@ func (s *Server) Close() { s.grpcServer.Stop() s.grpcServer = nil } + if s.autoIDService != nil { + s.autoIDService.Close() + } metrics.ServerEventCounter.WithLabelValues(metrics.EventClose).Inc() } From 5d68692b325ebc01381edcdbca1dca0e567272fd Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 11:06:40 +0800 Subject: [PATCH 03/16] update go.mod --- go.mod | 4 +--- go.sum | 11 +++++++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 32ba9ed3ae0b4..78cfdfba6f2e0 100644 --- a/go.mod +++ b/go.mod @@ -67,7 +67,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172 + github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9 github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20221013075951-a5c7c039c6c3 @@ -251,5 +251,3 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) - -replace github.com/pingcap/kvproto => github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 diff --git a/go.sum b/go.sum index 9920da6bd84ce..5fd7041c28757 100644 --- a/go.sum +++ b/go.sum @@ -338,6 +338,7 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -366,6 +367,7 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= +github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -772,6 +774,10 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= +github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= +github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9 h1:ho5XUD8DVCnkpEj8oiTR57FXDTXnH6znyLe0gyrtzKk= +github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -923,8 +929,6 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 h1:+sVrTp/1hSRuQM+dLDl4OnjqPANDXuNFZu+EhheAWuw= -github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c h1:NvQHWk0GeXSLEBbmGMPnDMc0to0a3ogzgIRbTKw8MHI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= @@ -1128,6 +1132,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1483,6 +1488,7 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1563,6 +1569,7 @@ google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljW google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= +google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= From 79833baaa3e5ea4e5873311401f11ec369b45444 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 11:08:45 +0800 Subject: [PATCH 04/16] update go.mod --- go.mod | 4 +--- go.sum | 11 +++++++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 32ba9ed3ae0b4..78cfdfba6f2e0 100644 --- a/go.mod +++ b/go.mod @@ -67,7 +67,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172 + github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9 github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20221013075951-a5c7c039c6c3 @@ -251,5 +251,3 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) - -replace github.com/pingcap/kvproto => github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 diff --git a/go.sum b/go.sum index 9920da6bd84ce..5fd7041c28757 100644 --- a/go.sum +++ b/go.sum @@ -338,6 +338,7 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -366,6 +367,7 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= +github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -772,6 +774,10 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= +github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= +github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9 h1:ho5XUD8DVCnkpEj8oiTR57FXDTXnH6znyLe0gyrtzKk= +github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -923,8 +929,6 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9 h1:+sVrTp/1hSRuQM+dLDl4OnjqPANDXuNFZu+EhheAWuw= -github.com/tiancaiamao/kvproto v0.0.0-20221101131513-753c3dd721c9/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c h1:NvQHWk0GeXSLEBbmGMPnDMc0to0a3ogzgIRbTKw8MHI= github.com/tikv/client-go/v2 v2.0.1-0.20221031063202-30e803b7082c/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= @@ -1128,6 +1132,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1483,6 +1488,7 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= @@ -1563,6 +1569,7 @@ google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljW google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= +google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= From 3982bb3ccb9d59f314a5894c31fbbe9a72d3d096 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 11:42:58 +0800 Subject: [PATCH 05/16] make bazel_prepare --- autoid_service/BUILD.bazel | 21 +++++++++++++++++++++ dumpling/export/BUILD.bazel | 1 + server/BUILD.bazel | 3 +++ 3 files changed, 25 insertions(+) create mode 100644 autoid_service/BUILD.bazel diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel new file mode 100644 index 0000000000000..7e29b826e7b29 --- /dev/null +++ b/autoid_service/BUILD.bazel @@ -0,0 +1,21 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "autoid_service", + srcs = ["autoid.go"], + importpath = "github.com/pingcap/tidb/autoid_service", + visibility = ["//visibility:public"], + deps = [ + "//kv", + "//meta", + "//metrics", + "//owner", + "//util/logutil", + "//util/mathutil", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/autoid", + "@io_etcd_go_etcd_client_v3//:client", + "@org_golang_google_grpc//:grpc", + "@org_uber_go_zap//:zap", + ], +) diff --git a/dumpling/export/BUILD.bazel b/dumpling/export/BUILD.bazel index 35f1f9925d43d..08cc7fe2e664e 100644 --- a/dumpling/export/BUILD.bazel +++ b/dumpling/export/BUILD.bazel @@ -62,6 +62,7 @@ go_library( "@io_etcd_go_etcd_client_v3//:client", "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", + "@org_uber_go_atomic//:atomic", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", ], diff --git a/server/BUILD.bazel b/server/BUILD.bazel index f33536d47ec23..d2b3206fec658 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -25,6 +25,7 @@ go_library( importpath = "github.com/pingcap/tidb/server", visibility = ["//visibility:public"], deps = [ + "//autoid_service", "//config", "//ddl", "//domain", @@ -56,6 +57,7 @@ go_library( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", + "//store", "//store/driver/error", "//store/gcworker", "//store/helper", @@ -92,6 +94,7 @@ go_library( "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_fn//:fn", + "@com_github_pingcap_kvproto//pkg/autoid", "@com_github_pingcap_kvproto//pkg/coprocessor", "@com_github_pingcap_kvproto//pkg/diagnosticspb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", From 64dd578308f4fd3b6ef9c1c68ee6221ff68a8473 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 11:44:23 +0800 Subject: [PATCH 06/16] make bazel_prepare --- autoid_service/BUILD.bazel | 13 +++++++++++++ dumpling/export/BUILD.bazel | 1 + meta/autoid/BUILD.bazel | 6 ++++++ 3 files changed, 20 insertions(+) create mode 100644 autoid_service/BUILD.bazel diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel new file mode 100644 index 0000000000000..84f4b7423a186 --- /dev/null +++ b/autoid_service/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "autoid_service", + srcs = ["autoid.go"], + importpath = "github.com/pingcap/tidb/autoid_service", + visibility = ["//visibility:public"], + deps = [ + "//kv", + "@com_github_pingcap_kvproto//pkg/autoid", + "@org_golang_google_grpc//:grpc", + ], +) diff --git a/dumpling/export/BUILD.bazel b/dumpling/export/BUILD.bazel index 35f1f9925d43d..08cc7fe2e664e 100644 --- a/dumpling/export/BUILD.bazel +++ b/dumpling/export/BUILD.bazel @@ -62,6 +62,7 @@ go_library( "@io_etcd_go_etcd_client_v3//:client", "@org_golang_x_exp//slices", "@org_golang_x_sync//errgroup", + "@org_uber_go_atomic//:atomic", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", ], diff --git a/meta/autoid/BUILD.bazel b/meta/autoid/BUILD.bazel index f5bb574907665..7490d65691e4c 100644 --- a/meta/autoid/BUILD.bazel +++ b/meta/autoid/BUILD.bazel @@ -4,12 +4,14 @@ go_library( name = "autoid", srcs = [ "autoid.go", + "autoid_service.go", "errors.go", "memid.go", ], importpath = "github.com/pingcap/tidb/meta/autoid", visibility = ["//visibility:public"], deps = [ + "//autoid_service", "//errno", "//kv", "//meta", @@ -24,8 +26,12 @@ go_library( "@com_github_opentracing_opentracing_go//:opentracing-go", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/autoid", "@com_github_tikv_client_go_v2//txnkv/txnsnapshot", "@com_github_tikv_client_go_v2//util", + "@io_etcd_go_etcd_client_v3//:client", + "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//credentials/insecure", "@org_uber_go_zap//:zap", ], ) From 324346a0f2306d5fd030dbf753b15b73dd410720 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 12:34:29 +0800 Subject: [PATCH 07/16] make bazel_prepare again --- DEPS.bzl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index e666f3a3fd300..e9ccd20301ea6 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2849,8 +2849,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:FYgKV9znRQmzVrrJDZ0gUfMIvKLAMU1tu1UKJib8bEQ=", - version = "v0.0.0-20221026112947-f8d61344b172", + sum = "h1:ho5XUD8DVCnkpEj8oiTR57FXDTXnH6znyLe0gyrtzKk=", + version = "v0.0.0-20221103025916-e7e21f0e9cd9", ) go_repository( name = "com_github_pingcap_log", From 4a2f1c9c59ec0e8c1c58458727b3dcba46c746c5 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 13:46:33 +0800 Subject: [PATCH 08/16] make golint happy --- autoid_service/autoid.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index 23d0555eb1497..2c2922fd58ce3 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -429,7 +429,10 @@ func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoi val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned) if req.Force { - val.forceRebase(ctx, s.store, req.DbID, req.TblID, req.Base, req.IsUnsigned) + err := val.forceRebase(ctx, s.store, req.DbID, req.TblID, req.Base, req.IsUnsigned) + if err != nil { + return nil, errors.Trace(err) + } } var err error From 3736de9d1914d6e9224f19df7b1f73224f3a1e24 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 14:14:07 +0800 Subject: [PATCH 09/16] make lint check happy --- autoid_service/autoid.go | 34 +++++++++++++++++++++++++++------- server/server.go | 1 - 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index 2c2922fd58ce3..7e0edb7ea3e63 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -1,3 +1,17 @@ +// Copyright 2022 PingCAP, Inc. +// +// 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 autoid import ( @@ -20,7 +34,7 @@ import ( ) var ( - ErrAutoincReadFailed = errors.New("auto increment action failed") + errAutoincReadFailed = errors.New("auto increment action failed") ) const ( @@ -74,7 +88,7 @@ func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage, tmpStep := int64(mathutil.Min(math.MaxUint64-uint64(newBase), uint64(nextStep))) // The global rest is not enough for alloc. if tmpStep < n1 { - return ErrAutoincReadFailed + return errAutoincReadFailed } newEnd, err1 = idAcc.Inc(tmpStep) return err1 @@ -83,7 +97,7 @@ func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage, return 0, 0, err } if uint64(newBase) == math.MaxUint64 { - return 0, 0, ErrAutoincReadFailed + return 0, 0, errAutoincReadFailed } alloc.base, alloc.end = newBase, newEnd } @@ -109,7 +123,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context, // Condition alloc.base+N1 > alloc.end will overflow when alloc.base + N1 > MaxInt64. So need this. if math.MaxInt64-alloc.base <= n1 { - return 0, 0, ErrAutoincReadFailed + return 0, 0, errAutoincReadFailed } // The local rest is not enough for allocN, skip it. @@ -135,7 +149,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context, tmpStep := mathutil.Min(math.MaxInt64-newBase, nextStep) // The global rest is not enough for alloc. if tmpStep < n1 { - return ErrAutoincReadFailed + return errAutoincReadFailed } newEnd, err1 = idAcc.Inc(tmpStep) return err1 @@ -144,7 +158,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context, return 0, 0, err } if newBase == math.MaxInt64 { - return 0, 0, ErrAutoincReadFailed + return 0, 0, errAutoincReadFailed } alloc.base, alloc.end = newBase, newEnd } @@ -221,6 +235,7 @@ func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, d return nil } +// Service implement the grpc AutoIDAlloc service, defined in kvproto/pkg/autoid. type Service struct { autoIDLock sync.Mutex autoIDMap map[autoIDKey]*autoIDValue @@ -229,6 +244,7 @@ type Service struct { store kv.Storage } +// New return a Service instance. func New(selfAddr string, etcdAddr []string, store kv.Storage) *Service { cli, err := clientv3.New(clientv3.Config{ Endpoints: etcdAddr, @@ -265,6 +281,7 @@ func (m *mockClient) Rebase(ctx context.Context, in *autoid.RebaseRequest, opts var global = make(map[string]*mockClient) +// MockForTest is used for testing, the UT test and unistore use this. func MockForTest(store kv.Storage) *mockClient { uuid := store.UUID() ret, ok := global[uuid] @@ -281,6 +298,7 @@ func MockForTest(store kv.Storage) *mockClient { return ret } +// Close closes the Service and clean up resource. func (s *Service) Close() { if s.leaderShip != nil { for k, v := range s.autoIDMap { @@ -332,7 +350,7 @@ func calcNeededBatchSize(base, n, increment, offset int64, isUnsigned bool) int6 const batch = 4000 -// AllocID implements gRPC PDServer. +// AllocAutoID implements gRPC AutoIDAlloc interface. func (s *Service) AllocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*autoid.AutoIDResponse, error) { var res *autoid.AutoIDResponse for { @@ -422,6 +440,8 @@ func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbI return nil } +// Rebase implements gRPC AutoIDAlloc interface. +// req.N = 0 is handled specially, it is used to return the current auto ID value. func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoid.RebaseResponse, error) { if s.leaderShip != nil && !s.leaderShip.IsOwner() { return nil, errors.New("not leader") diff --git a/server/server.go b/server/server.go index 429c362c3da2c..408594bb696b8 100644 --- a/server/server.go +++ b/server/server.go @@ -37,7 +37,6 @@ import ( "math/rand" "net" "net/http" //nolint:goimports - // For pprof _ "net/http/pprof" // #nosec G108 "os" From 0688e4ec727c3042823c3ee3fa71ba02a0c39e34 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 15:10:32 +0800 Subject: [PATCH 10/16] make bazel_prepare --- DEPS.bzl | 197 +++++++++++++++++++++++------------ autoid_service/autoid.go | 15 +++ ddl/schematracker/checker.go | 3 + 3 files changed, 151 insertions(+), 64 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index be3c5c4c6e47f..ad289dc36e1db 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -524,8 +524,8 @@ def go_deps(): name = "com_github_cockroachdb_pebble", build_file_proto_mode = "disable_global", importpath = "github.com/cockroachdb/pebble", - sum = "h1:Igd6YmtOZ77EgLAIaE9+mHl7+sAKaZ5m4iMI0Dz/J2A=", - version = "v0.0.0-20210719141320-8c3bd06debb5", + sum = "h1:snjwkhKc/ZtYIC/hg6UoT5PrhXcZmCsaB+z0bonMDcU=", + version = "v0.0.0-20220415182917-06c9d3be25b3", ) go_repository( name = "com_github_cockroachdb_redact", @@ -694,8 +694,8 @@ def go_deps(): name = "com_github_danjacques_gofslock", build_file_proto_mode = "disable_global", importpath = "github.com/danjacques/gofslock", - sum = "h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w=", - version = "v0.0.0-20191023191349-0a45f885bc37", + sum = "h1:+4P40F8AqFAW4/ft2WXiZXrgtRbS8RLb61D8e6NcMw0=", + version = "v0.0.0-20220131014315-6e321f4509c8", ) go_repository( name = "com_github_data_dog_go_sqlmock", @@ -708,8 +708,8 @@ def go_deps(): name = "com_github_datadog_zstd", build_file_proto_mode = "disable_global", importpath = "github.com/DataDog/zstd", - sum = "h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ=", - version = "v1.4.5", + sum = "h1:++HGU87uq9UsSTlFeiOV9uZR3NpYkndUXeYyLv2DTc8=", + version = "v1.4.6-0.20210211175136-c6db21d202f4", ) go_repository( name = "com_github_davecgh_go_spew", @@ -795,8 +795,8 @@ def go_deps(): name = "com_github_eapache_go_resiliency", build_file_proto_mode = "disable_global", importpath = "github.com/eapache/go-resiliency", - sum = "h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q=", - version = "v1.2.0", + sum = "h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0=", + version = "v1.3.0", ) go_repository( name = "com_github_eapache_go_xerial_snappy", @@ -845,8 +845,8 @@ def go_deps(): name = "com_github_envoyproxy_go_control_plane", build_file_proto_mode = "disable_global", importpath = "github.com/envoyproxy/go-control-plane", - sum = "h1:fP+fF0up6oPY49OrjPrhIJ8yQfdIM85NXMLkMg1EXVs=", - version = "v0.9.10-0.20210907150352-cf90f659a021", + sum = "h1:xvqufLtNVwAhN8NMyWklVgxnWohi+wtMGQMhtxexlm0=", + version = "v0.10.2-0.20220325020618-49ff273808a1", ) go_repository( name = "com_github_envoyproxy_protoc_gen_validate", @@ -1490,12 +1490,20 @@ def go_deps(): sum = "h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=", version = "v1.3.0", ) + go_repository( + name = "com_github_googleapis_enterprise_certificate_proxy", + build_file_proto_mode = "disable", + importpath = "github.com/googleapis/enterprise-certificate-proxy", + sum = "h1:7MYGT2XEMam7Mtzv1yDUYXANedWvwk3HKkR3MyGowy8=", + version = "v0.0.0-20220520183353-fd19c99a87aa", + ) + go_repository( name = "com_github_googleapis_gax_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/googleapis/gax-go/v2", - sum = "h1:s7jOdKSaksJVOxE0Y/S32otcfiP+UQ0cL8/GTKaONwE=", - version = "v2.2.0", + sum = "h1:dS9eYAjhrE2RjmzYw2XAPvcXfmcQLtFEQWn0CR82awk=", + version = "v2.4.0", ) go_repository( name = "com_github_googleapis_gnostic", @@ -1504,6 +1512,14 @@ def go_deps(): sum = "h1:l6N3VoaVzTncYYW+9yOz2LJJammFZGBO13sqgEhpy9g=", version = "v0.2.0", ) + go_repository( + name = "com_github_googleapis_go_type_adapters", + build_file_proto_mode = "disable", + importpath = "github.com/googleapis/go-type-adapters", + sum = "h1:9XdMn+d/G57qq1s8dNc5IesGCXHf6V2HZ2JwRxfA2tA=", + version = "v1.0.0", + ) + go_repository( name = "com_github_gophercloud_gophercloud", build_file_proto_mode = "disable", @@ -1720,8 +1736,8 @@ def go_deps(): name = "com_github_hashicorp_go_uuid", build_file_proto_mode = "disable_global", importpath = "github.com/hashicorp/go-uuid", - sum = "h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE=", - version = "v1.0.2", + sum = "h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8=", + version = "v1.0.3", ) go_repository( name = "com_github_hashicorp_go_version", @@ -1905,8 +1921,8 @@ def go_deps(): name = "com_github_jcmturner_gofork", build_file_proto_mode = "disable_global", importpath = "github.com/jcmturner/gofork", - sum = "h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8=", - version = "v1.0.0", + sum = "h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg=", + version = "v1.7.6", ) go_repository( name = "com_github_jcmturner_goidentity_v6", @@ -1919,8 +1935,8 @@ def go_deps(): name = "com_github_jcmturner_gokrb5_v8", build_file_proto_mode = "disable_global", importpath = "github.com/jcmturner/gokrb5/v8", - sum = "h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA=", - version = "v8.4.2", + sum = "h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8=", + version = "v8.4.3", ) go_repository( name = "com_github_jcmturner_rpc_v2", @@ -2004,8 +2020,8 @@ def go_deps(): name = "com_github_jonboulle_clockwork", build_file_proto_mode = "disable_global", importpath = "github.com/jonboulle/clockwork", - sum = "h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ=", - version = "v0.2.2", + sum = "h1:9BSCMi8C+0qdApAp4auwX0RkLGUjs956h0EkuQymUhg=", + version = "v0.3.0", ) go_repository( name = "com_github_jpillora_backoff", @@ -2135,8 +2151,8 @@ def go_deps(): name = "com_github_klauspost_compress", build_file_proto_mode = "disable_global", importpath = "github.com/klauspost/compress", - sum = "h1:y9FcTHGyrebwfP0ZZqFiaxTaiDnUrGkJkI+f583BL1A=", - version = "v1.15.1", + sum = "h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY=", + version = "v1.15.9", ) go_repository( name = "com_github_klauspost_cpuid", @@ -2273,8 +2289,8 @@ def go_deps(): name = "com_github_lufia_plan9stats", build_file_proto_mode = "disable_global", importpath = "github.com/lufia/plan9stats", - sum = "h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4=", - version = "v0.0.0-20211012122336-39d0f177ccd0", + sum = "h1:m2s7Fwo4OwmcheIWUc/Nw9/MZ0eFtP3to0ovTpqOiCQ=", + version = "v0.0.0-20220326011226-f1430873d8db", ) go_repository( name = "com_github_magiconair_properties", @@ -2667,8 +2683,8 @@ def go_deps(): name = "com_github_opentracing_basictracer_go", build_file_proto_mode = "disable_global", importpath = "github.com/opentracing/basictracer-go", - sum = "h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo=", - version = "v1.0.0", + sum = "h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0=", + version = "v1.1.0", ) go_repository( name = "com_github_opentracing_contrib_go_stdlib", @@ -2779,6 +2795,14 @@ def go_deps(): sum = "h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM=", version = "v2.6.1+incompatible", ) + go_repository( + name = "com_github_pierrec_lz4_v4", + build_file_proto_mode = "disable", + importpath = "github.com/pierrec/lz4/v4", + sum = "h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0=", + version = "v4.1.15", + ) + go_repository( name = "com_github_pingcap_badger", build_file_proto_mode = "disable_global", @@ -2790,8 +2814,8 @@ def go_deps(): name = "com_github_pingcap_check", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/check", - sum = "h1:iRtOAQ6FXkY/BGvst3CDfTva4nTqh6CL8WXvanLdbu0=", - version = "v0.0.0-20191107115940-caf2b9e6ccf4", + sum = "h1:HVl5539r48eA+uDuX/ziBmQCxzT1pGrzWbKuXT46Bq0=", + version = "v0.0.0-20211026125417-57bd13f7b5f0", ) go_repository( name = "com_github_pingcap_errors", @@ -2825,8 +2849,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:FYgKV9znRQmzVrrJDZ0gUfMIvKLAMU1tu1UKJib8bEQ=", - version = "v0.0.0-20221026112947-f8d61344b172", + sum = "h1:ho5XUD8DVCnkpEj8oiTR57FXDTXnH6znyLe0gyrtzKk=", + version = "v0.0.0-20221103025916-e7e21f0e9cd9", ) go_repository( name = "com_github_pingcap_log", @@ -2856,6 +2880,14 @@ def go_deps(): sum = "h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=", version = "v0.0.0-20180916011732-0a3d74bf9ce4", ) + go_repository( + name = "com_github_pkg_diff", + build_file_proto_mode = "disable", + importpath = "github.com/pkg/diff", + sum = "h1:aoZm08cpOy4WuID//EZDgcC4zIxODThtZNPirFr42+A=", + version = "v0.0.0-20210226163009-20ebb0f2a09e", + ) + go_repository( name = "com_github_pkg_errors", build_file_proto_mode = "disable_global", @@ -2989,8 +3021,8 @@ def go_deps(): name = "com_github_remyoudompheng_bigfft", build_file_proto_mode = "disable_global", importpath = "github.com/remyoudompheng/bigfft", - sum = "h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk=", - version = "v0.0.0-20200410134404-eec4a21b6bb0", + sum = "h1:tEkEyxYeZ43TR55QU/hsIt9aRGBxbgGuz9CGykjvogY=", + version = "v0.0.0-20220927061507-ef77025ab5aa", ) go_repository( name = "com_github_rivo_uniseg", @@ -3022,8 +3054,8 @@ def go_deps(): name = "com_github_rogpeppe_go_internal", build_file_proto_mode = "disable_global", importpath = "github.com/rogpeppe/go-internal", - sum = "h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k=", - version = "v1.6.1", + sum = "h1:FCbCCtXNOY3UtUuHUYaghJg4y7Fd14rXifAYUAtL9R8=", + version = "v1.8.0", ) go_repository( name = "com_github_rubyist_circuitbreaker", @@ -3169,8 +3201,8 @@ def go_deps(): name = "com_github_shopify_sarama", build_file_proto_mode = "disable_global", importpath = "github.com/Shopify/sarama", - sum = "h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE=", - version = "v1.29.0", + sum = "h1:0OJs3eCcnezkWniVjwBbCJVaa0B1k7ImCRS3WN6NsSk=", + version = "v1.36.0", ) go_repository( name = "com_github_shopify_toxiproxy", @@ -3179,6 +3211,14 @@ def go_deps(): sum = "h1:TKdv8HiTLgE5wdJuEML90aBgNWsokNbMijUGhmcoBJc=", version = "v2.1.4+incompatible", ) + go_repository( + name = "com_github_shopify_toxiproxy_v2", + build_file_proto_mode = "disable", + importpath = "github.com/Shopify/toxiproxy/v2", + sum = "h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64=", + version = "v2.4.0", + ) + go_repository( name = "com_github_shopspring_decimal", build_file_proto_mode = "disable", @@ -3501,8 +3541,8 @@ def go_deps(): name = "com_github_uber_jaeger_client_go", build_file_proto_mode = "disable_global", importpath = "github.com/uber/jaeger-client-go", - sum = "h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM=", - version = "v2.22.1+incompatible", + sum = "h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o=", + version = "v2.30.0+incompatible", ) go_repository( name = "com_github_uber_jaeger_lib", @@ -3539,6 +3579,13 @@ def go_deps(): sum = "h1:hh+/cpIcopyMYbZNVov9iSxvJU3OYQg78Sfaqzi/CzI=", version = "v0.0.5", ) + go_repository( + name = "com_github_urfave_cli_v2", + build_file_proto_mode = "disable", + importpath = "github.com/urfave/cli/v2", + sum = "h1:qph92Y649prgesehzOrQjdWyxFOp/QVM+6imKHad91M=", + version = "v2.3.0", + ) go_repository( name = "com_github_urfave_negroni", @@ -3613,6 +3660,28 @@ def go_deps(): sum = "h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY=", version = "v0.0.0-20161129230411-ed8402a42d5f", ) + go_repository( + name = "com_github_xdg_go_pbkdf2", + build_file_proto_mode = "disable", + importpath = "github.com/xdg-go/pbkdf2", + sum = "h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c=", + version = "v1.0.0", + ) + go_repository( + name = "com_github_xdg_go_scram", + build_file_proto_mode = "disable", + importpath = "github.com/xdg-go/scram", + sum = "h1:VOMT+81stJgXW3CpHyqHN3AXDYIMsx56mEFrB37Mb/E=", + version = "v1.1.1", + ) + go_repository( + name = "com_github_xdg_go_stringprep", + build_file_proto_mode = "disable", + importpath = "github.com/xdg-go/stringprep", + sum = "h1:kdwGpVNwPFtjs98xCGkHjQtGKh86rDcRZN17QEMCOIs=", + version = "v1.0.3", + ) + go_repository( name = "com_github_xdg_scram", build_file_proto_mode = "disable_global", @@ -3659,8 +3728,8 @@ def go_deps(): name = "com_github_xitongsys_parquet_go", build_file_proto_mode = "disable_global", importpath = "github.com/xitongsys/parquet-go", - sum = "h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs=", - version = "v1.5.5-0.20201110004701-b09c49d6d457", + sum = "h1:j6YrTVZdQx5yywJLIOklZcKVsCoSD1tqOVRXyTBFSjs=", + version = "v1.6.0", ) go_repository( name = "com_github_xitongsys_parquet_go_source", @@ -3746,8 +3815,8 @@ def go_deps(): name = "com_google_cloud_go", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go", - sum = "h1:t9Iw5QH5v4XtlEQaCtUY7x6sCABps8sW0acw7e2WQ6Y=", - version = "v0.100.2", + sum = "h1:DAq3r8y4mDgyB/ZPJ9v/5VJNqjgJAxTn6ZYLlUywOu8=", + version = "v0.102.0", ) go_repository( name = "com_google_cloud_go_bigquery", @@ -3760,8 +3829,8 @@ def go_deps(): name = "com_google_cloud_go_compute", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/compute", - sum = "h1:b1zWmYuuHz7gO9kDcM/EpHGr06UgsYNRpNJzI2kFiLM=", - version = "v1.5.0", + sum = "h1:v/k9Eueb8aAJ0vZuxKMrgm6kPhCLZU9HxFU+AFDs9Uk=", + version = "v1.7.0", ) go_repository( name = "com_google_cloud_go_datastore", @@ -3781,8 +3850,8 @@ def go_deps(): name = "com_google_cloud_go_iam", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/iam", - sum = "h1:4CapQyNFjiksks1/x7jsvsygFPhihslYk5GptIrlX68=", - version = "v0.1.1", + sum = "h1:exkAomrVUuzx9kWFI1wm3KI0uoDeUFPB4kKGzx6x+Gc=", + version = "v0.3.0", ) go_repository( name = "com_google_cloud_go_pubsub", @@ -3795,8 +3864,8 @@ def go_deps(): name = "com_google_cloud_go_storage", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/storage", - sum = "h1:HwnT2u2D309SFDHQII6m18HlrCi3jAXhUMTLOWXYH14=", - version = "v1.21.0", + sum = "h1:F6IlQJZrZM++apn9V5/VfS3gbTUYg98PS3EMQAzqtfg=", + version = "v1.22.1", ) go_repository( name = "com_shuralyov_dmitri_gpu_mtl", @@ -4223,8 +4292,8 @@ def go_deps(): name = "org_golang_google_api", build_file_proto_mode = "disable_global", importpath = "google.golang.org/api", - sum = "h1:ExR2D+5TYIrMphWgs5JCgwRhEDlPDXXrLwHHMgPHTXE=", - version = "v0.74.0", + sum = "h1:NMB9J4cCxs9xEm+1Z9QiO3eFvn7EnQj3Eo3hN6ugVlg=", + version = "v0.84.0", ) go_repository( name = "org_golang_google_appengine", @@ -4237,15 +4306,15 @@ def go_deps(): name = "org_golang_google_genproto", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto", - sum = "h1:0m9wktIpOxGw+SSKmydXWB3Z3GTfcPP6+q75HCQa6HI=", - version = "v0.0.0-20220324131243-acbaeb5b85eb", + sum = "h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM=", + version = "v0.0.0-20220719170305-83ca9fad585f", ) go_repository( name = "org_golang_google_grpc", build_file_proto_mode = "disable_global", importpath = "google.golang.org/grpc", - sum = "h1:NEpgUqV3Z+ZjkqMsxMg11IaDrXY4RY6CQukSGK0uI1M=", - version = "v1.45.0", + sum = "h1:rQOsyJ/8+ufEDJd/Gdsz7HG220Mh9HAhFHRGnIjda0w=", + version = "v1.48.0", ) go_repository( name = "org_golang_google_grpc_cmd_protoc_gen_go_grpc", @@ -4265,8 +4334,8 @@ def go_deps(): name = "org_golang_x_crypto", build_file_proto_mode = "disable_global", importpath = "golang.org/x/crypto", - sum = "h1:kUhD7nTDoI3fVd9G4ORWrbV5NY0liEs/Jg2pv5f+bBA=", - version = "v0.0.0-20220411220226-7b82a4e95df4", + sum = "h1:zuSxTR4o9y82ebqCUJYNGJbGPo6sKVl54f/TVDObg1c=", + version = "v0.0.0-20220722155217-630584e8d5aa", ) go_repository( name = "org_golang_x_exp", @@ -4315,15 +4384,15 @@ def go_deps(): name = "org_golang_x_net", build_file_proto_mode = "disable_global", importpath = "golang.org/x/net", - sum = "h1:PxfKdU9lEEDYjdIzOtC4qFWgkU2rGHdKlKowJSMN9h0=", - version = "v0.0.0-20220722155237-a158d28d115b", + sum = "h1:3dYNDff0VT5xj+mbj2XucFst9WKk6PdGOrb9n+SbIvw=", + version = "v0.0.0-20220809184613-07c6da5e1ced", ) go_repository( name = "org_golang_x_oauth2", build_file_proto_mode = "disable_global", importpath = "golang.org/x/oauth2", - sum = "h1:OSnWWcOd/CtWQC2cYSBgbTSJv3ciqd8r54ySIW2y3RE=", - version = "v0.0.0-20220411215720-9780585627b5", + sum = "h1:8tDJ3aechhddbdPAxpycgXHJRMLpk/Ab+aa4OgdN5/g=", + version = "v0.0.0-20220608161450-d0670ef3b1eb", ) go_repository( name = "org_golang_x_sync", @@ -4343,8 +4412,8 @@ def go_deps(): name = "org_golang_x_term", build_file_proto_mode = "disable_global", importpath = "golang.org/x/term", - sum = "h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY=", - version = "v0.0.0-20210927222741-03fcf44c2211", + sum = "h1:EH1Deb8WZJ0xc0WK//leUHXcX9aLE5SymusoTmMZye8=", + version = "v0.0.0-20220411215600-e5f449aeb171", ) go_repository( name = "org_golang_x_text", @@ -4371,8 +4440,8 @@ def go_deps(): name = "org_golang_x_xerrors", build_file_proto_mode = "disable_global", importpath = "golang.org/x/xerrors", - sum = "h1:GGU+dLjvlC3qDwqYgL6UgRmHXhOOgns0bZu2Ty5mm6U=", - version = "v0.0.0-20220411194840-2f41105eb62f", + sum = "h1:uF6paiQQebLeSXkrTqHqz0MXhXXS1KgF41eUdBNvxK0=", + version = "v0.0.0-20220609144429-65e65417b02f", ) go_repository( name = "org_gonum_v1_gonum", diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index eb23a0edefdc7..700d0c5bb262b 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -1,3 +1,17 @@ +// Copyright 2022 PingCAP, Inc. +// +// 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 autoid import ( @@ -21,6 +35,7 @@ func (m *mockClient) Rebase(ctx context.Context, in *autoid.RebaseRequest, opts var global = make(map[string]*mockClient) +// MockForTest is used for testing, the UT test and unistore use this. func MockForTest(store kv.Storage) *mockClient { uuid := store.UUID() ret, ok := global[uuid] diff --git a/ddl/schematracker/checker.go b/ddl/schematracker/checker.go index 018c93f6f416e..b1533d0246fb1 100644 --- a/ddl/schematracker/checker.go +++ b/ddl/schematracker/checker.go @@ -560,14 +560,17 @@ type StorageDDLInjector struct { var _ kv.EtcdBackend = StorageDDLInjector{} +// EtcdAddrs implements the kv.EtcdBackend interface. func (s StorageDDLInjector) EtcdAddrs() ([]string, error) { return s.EtcdBackend.EtcdAddrs() } +// TLSConfig implements the kv.EtcdBackend interface. func (s StorageDDLInjector) TLSConfig() *tls.Config { return s.EtcdBackend.TLSConfig() } +// StartGCWorker implements the kv.EtcdBackend interface. func (s StorageDDLInjector) StartGCWorker() error { return s.EtcdBackend.StartGCWorker() } From c165725147137dbfb0c46c5506d7ce94126a6cb5 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 16:22:37 +0800 Subject: [PATCH 11/16] make bazel_prepare --- DEPS.bzl | 124 +++++++++++++++++++++++++++---------------------------- 1 file changed, 62 insertions(+), 62 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index ad289dc36e1db..e9ccd20301ea6 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -524,8 +524,8 @@ def go_deps(): name = "com_github_cockroachdb_pebble", build_file_proto_mode = "disable_global", importpath = "github.com/cockroachdb/pebble", - sum = "h1:snjwkhKc/ZtYIC/hg6UoT5PrhXcZmCsaB+z0bonMDcU=", - version = "v0.0.0-20220415182917-06c9d3be25b3", + sum = "h1:Igd6YmtOZ77EgLAIaE9+mHl7+sAKaZ5m4iMI0Dz/J2A=", + version = "v0.0.0-20210719141320-8c3bd06debb5", ) go_repository( name = "com_github_cockroachdb_redact", @@ -694,8 +694,8 @@ def go_deps(): name = "com_github_danjacques_gofslock", build_file_proto_mode = "disable_global", importpath = "github.com/danjacques/gofslock", - sum = "h1:+4P40F8AqFAW4/ft2WXiZXrgtRbS8RLb61D8e6NcMw0=", - version = "v0.0.0-20220131014315-6e321f4509c8", + sum = "h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w=", + version = "v0.0.0-20191023191349-0a45f885bc37", ) go_repository( name = "com_github_data_dog_go_sqlmock", @@ -708,8 +708,8 @@ def go_deps(): name = "com_github_datadog_zstd", build_file_proto_mode = "disable_global", importpath = "github.com/DataDog/zstd", - sum = "h1:++HGU87uq9UsSTlFeiOV9uZR3NpYkndUXeYyLv2DTc8=", - version = "v1.4.6-0.20210211175136-c6db21d202f4", + sum = "h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ=", + version = "v1.4.5", ) go_repository( name = "com_github_davecgh_go_spew", @@ -795,8 +795,8 @@ def go_deps(): name = "com_github_eapache_go_resiliency", build_file_proto_mode = "disable_global", importpath = "github.com/eapache/go-resiliency", - sum = "h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0=", - version = "v1.3.0", + sum = "h1:v7g92e/KSN71Rq7vSThKaWIq68fL4YHvWyiUKorFR1Q=", + version = "v1.2.0", ) go_repository( name = "com_github_eapache_go_xerial_snappy", @@ -845,8 +845,8 @@ def go_deps(): name = "com_github_envoyproxy_go_control_plane", build_file_proto_mode = "disable_global", importpath = "github.com/envoyproxy/go-control-plane", - sum = "h1:xvqufLtNVwAhN8NMyWklVgxnWohi+wtMGQMhtxexlm0=", - version = "v0.10.2-0.20220325020618-49ff273808a1", + sum = "h1:fP+fF0up6oPY49OrjPrhIJ8yQfdIM85NXMLkMg1EXVs=", + version = "v0.9.10-0.20210907150352-cf90f659a021", ) go_repository( name = "com_github_envoyproxy_protoc_gen_validate", @@ -1502,8 +1502,8 @@ def go_deps(): name = "com_github_googleapis_gax_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/googleapis/gax-go/v2", - sum = "h1:dS9eYAjhrE2RjmzYw2XAPvcXfmcQLtFEQWn0CR82awk=", - version = "v2.4.0", + sum = "h1:s7jOdKSaksJVOxE0Y/S32otcfiP+UQ0cL8/GTKaONwE=", + version = "v2.2.0", ) go_repository( name = "com_github_googleapis_gnostic", @@ -1736,8 +1736,8 @@ def go_deps(): name = "com_github_hashicorp_go_uuid", build_file_proto_mode = "disable_global", importpath = "github.com/hashicorp/go-uuid", - sum = "h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8=", - version = "v1.0.3", + sum = "h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE=", + version = "v1.0.2", ) go_repository( name = "com_github_hashicorp_go_version", @@ -1921,8 +1921,8 @@ def go_deps(): name = "com_github_jcmturner_gofork", build_file_proto_mode = "disable_global", importpath = "github.com/jcmturner/gofork", - sum = "h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg=", - version = "v1.7.6", + sum = "h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8=", + version = "v1.0.0", ) go_repository( name = "com_github_jcmturner_goidentity_v6", @@ -1935,8 +1935,8 @@ def go_deps(): name = "com_github_jcmturner_gokrb5_v8", build_file_proto_mode = "disable_global", importpath = "github.com/jcmturner/gokrb5/v8", - sum = "h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8=", - version = "v8.4.3", + sum = "h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA=", + version = "v8.4.2", ) go_repository( name = "com_github_jcmturner_rpc_v2", @@ -2020,8 +2020,8 @@ def go_deps(): name = "com_github_jonboulle_clockwork", build_file_proto_mode = "disable_global", importpath = "github.com/jonboulle/clockwork", - sum = "h1:9BSCMi8C+0qdApAp4auwX0RkLGUjs956h0EkuQymUhg=", - version = "v0.3.0", + sum = "h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ=", + version = "v0.2.2", ) go_repository( name = "com_github_jpillora_backoff", @@ -2151,8 +2151,8 @@ def go_deps(): name = "com_github_klauspost_compress", build_file_proto_mode = "disable_global", importpath = "github.com/klauspost/compress", - sum = "h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY=", - version = "v1.15.9", + sum = "h1:y9FcTHGyrebwfP0ZZqFiaxTaiDnUrGkJkI+f583BL1A=", + version = "v1.15.1", ) go_repository( name = "com_github_klauspost_cpuid", @@ -2289,8 +2289,8 @@ def go_deps(): name = "com_github_lufia_plan9stats", build_file_proto_mode = "disable_global", importpath = "github.com/lufia/plan9stats", - sum = "h1:m2s7Fwo4OwmcheIWUc/Nw9/MZ0eFtP3to0ovTpqOiCQ=", - version = "v0.0.0-20220326011226-f1430873d8db", + sum = "h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4=", + version = "v0.0.0-20211012122336-39d0f177ccd0", ) go_repository( name = "com_github_magiconair_properties", @@ -2683,8 +2683,8 @@ def go_deps(): name = "com_github_opentracing_basictracer_go", build_file_proto_mode = "disable_global", importpath = "github.com/opentracing/basictracer-go", - sum = "h1:Oa1fTSBvAl8pa3U+IJYqrKm0NALwH9OsgwOqDv4xJW0=", - version = "v1.1.0", + sum = "h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo=", + version = "v1.0.0", ) go_repository( name = "com_github_opentracing_contrib_go_stdlib", @@ -2814,8 +2814,8 @@ def go_deps(): name = "com_github_pingcap_check", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/check", - sum = "h1:HVl5539r48eA+uDuX/ziBmQCxzT1pGrzWbKuXT46Bq0=", - version = "v0.0.0-20211026125417-57bd13f7b5f0", + sum = "h1:iRtOAQ6FXkY/BGvst3CDfTva4nTqh6CL8WXvanLdbu0=", + version = "v0.0.0-20191107115940-caf2b9e6ccf4", ) go_repository( name = "com_github_pingcap_errors", @@ -3021,8 +3021,8 @@ def go_deps(): name = "com_github_remyoudompheng_bigfft", build_file_proto_mode = "disable_global", importpath = "github.com/remyoudompheng/bigfft", - sum = "h1:tEkEyxYeZ43TR55QU/hsIt9aRGBxbgGuz9CGykjvogY=", - version = "v0.0.0-20220927061507-ef77025ab5aa", + sum = "h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk=", + version = "v0.0.0-20200410134404-eec4a21b6bb0", ) go_repository( name = "com_github_rivo_uniseg", @@ -3054,8 +3054,8 @@ def go_deps(): name = "com_github_rogpeppe_go_internal", build_file_proto_mode = "disable_global", importpath = "github.com/rogpeppe/go-internal", - sum = "h1:FCbCCtXNOY3UtUuHUYaghJg4y7Fd14rXifAYUAtL9R8=", - version = "v1.8.0", + sum = "h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k=", + version = "v1.6.1", ) go_repository( name = "com_github_rubyist_circuitbreaker", @@ -3201,8 +3201,8 @@ def go_deps(): name = "com_github_shopify_sarama", build_file_proto_mode = "disable_global", importpath = "github.com/Shopify/sarama", - sum = "h1:0OJs3eCcnezkWniVjwBbCJVaa0B1k7ImCRS3WN6NsSk=", - version = "v1.36.0", + sum = "h1:ARid8o8oieau9XrHI55f/L3EoRAhm9px6sonbD7yuUE=", + version = "v1.29.0", ) go_repository( name = "com_github_shopify_toxiproxy", @@ -3541,8 +3541,8 @@ def go_deps(): name = "com_github_uber_jaeger_client_go", build_file_proto_mode = "disable_global", importpath = "github.com/uber/jaeger-client-go", - sum = "h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o=", - version = "v2.30.0+incompatible", + sum = "h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM=", + version = "v2.22.1+incompatible", ) go_repository( name = "com_github_uber_jaeger_lib", @@ -3728,8 +3728,8 @@ def go_deps(): name = "com_github_xitongsys_parquet_go", build_file_proto_mode = "disable_global", importpath = "github.com/xitongsys/parquet-go", - sum = "h1:j6YrTVZdQx5yywJLIOklZcKVsCoSD1tqOVRXyTBFSjs=", - version = "v1.6.0", + sum = "h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs=", + version = "v1.5.5-0.20201110004701-b09c49d6d457", ) go_repository( name = "com_github_xitongsys_parquet_go_source", @@ -3815,8 +3815,8 @@ def go_deps(): name = "com_google_cloud_go", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go", - sum = "h1:DAq3r8y4mDgyB/ZPJ9v/5VJNqjgJAxTn6ZYLlUywOu8=", - version = "v0.102.0", + sum = "h1:t9Iw5QH5v4XtlEQaCtUY7x6sCABps8sW0acw7e2WQ6Y=", + version = "v0.100.2", ) go_repository( name = "com_google_cloud_go_bigquery", @@ -3829,8 +3829,8 @@ def go_deps(): name = "com_google_cloud_go_compute", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/compute", - sum = "h1:v/k9Eueb8aAJ0vZuxKMrgm6kPhCLZU9HxFU+AFDs9Uk=", - version = "v1.7.0", + sum = "h1:b1zWmYuuHz7gO9kDcM/EpHGr06UgsYNRpNJzI2kFiLM=", + version = "v1.5.0", ) go_repository( name = "com_google_cloud_go_datastore", @@ -3850,8 +3850,8 @@ def go_deps(): name = "com_google_cloud_go_iam", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/iam", - sum = "h1:exkAomrVUuzx9kWFI1wm3KI0uoDeUFPB4kKGzx6x+Gc=", - version = "v0.3.0", + sum = "h1:4CapQyNFjiksks1/x7jsvsygFPhihslYk5GptIrlX68=", + version = "v0.1.1", ) go_repository( name = "com_google_cloud_go_pubsub", @@ -3864,8 +3864,8 @@ def go_deps(): name = "com_google_cloud_go_storage", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/storage", - sum = "h1:F6IlQJZrZM++apn9V5/VfS3gbTUYg98PS3EMQAzqtfg=", - version = "v1.22.1", + sum = "h1:HwnT2u2D309SFDHQII6m18HlrCi3jAXhUMTLOWXYH14=", + version = "v1.21.0", ) go_repository( name = "com_shuralyov_dmitri_gpu_mtl", @@ -4292,8 +4292,8 @@ def go_deps(): name = "org_golang_google_api", build_file_proto_mode = "disable_global", importpath = "google.golang.org/api", - sum = "h1:NMB9J4cCxs9xEm+1Z9QiO3eFvn7EnQj3Eo3hN6ugVlg=", - version = "v0.84.0", + sum = "h1:ExR2D+5TYIrMphWgs5JCgwRhEDlPDXXrLwHHMgPHTXE=", + version = "v0.74.0", ) go_repository( name = "org_golang_google_appengine", @@ -4306,15 +4306,15 @@ def go_deps(): name = "org_golang_google_genproto", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto", - sum = "h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM=", - version = "v0.0.0-20220719170305-83ca9fad585f", + sum = "h1:0m9wktIpOxGw+SSKmydXWB3Z3GTfcPP6+q75HCQa6HI=", + version = "v0.0.0-20220324131243-acbaeb5b85eb", ) go_repository( name = "org_golang_google_grpc", build_file_proto_mode = "disable_global", importpath = "google.golang.org/grpc", - sum = "h1:rQOsyJ/8+ufEDJd/Gdsz7HG220Mh9HAhFHRGnIjda0w=", - version = "v1.48.0", + sum = "h1:NEpgUqV3Z+ZjkqMsxMg11IaDrXY4RY6CQukSGK0uI1M=", + version = "v1.45.0", ) go_repository( name = "org_golang_google_grpc_cmd_protoc_gen_go_grpc", @@ -4334,8 +4334,8 @@ def go_deps(): name = "org_golang_x_crypto", build_file_proto_mode = "disable_global", importpath = "golang.org/x/crypto", - sum = "h1:zuSxTR4o9y82ebqCUJYNGJbGPo6sKVl54f/TVDObg1c=", - version = "v0.0.0-20220722155217-630584e8d5aa", + sum = "h1:kUhD7nTDoI3fVd9G4ORWrbV5NY0liEs/Jg2pv5f+bBA=", + version = "v0.0.0-20220411220226-7b82a4e95df4", ) go_repository( name = "org_golang_x_exp", @@ -4384,15 +4384,15 @@ def go_deps(): name = "org_golang_x_net", build_file_proto_mode = "disable_global", importpath = "golang.org/x/net", - sum = "h1:3dYNDff0VT5xj+mbj2XucFst9WKk6PdGOrb9n+SbIvw=", - version = "v0.0.0-20220809184613-07c6da5e1ced", + sum = "h1:PxfKdU9lEEDYjdIzOtC4qFWgkU2rGHdKlKowJSMN9h0=", + version = "v0.0.0-20220722155237-a158d28d115b", ) go_repository( name = "org_golang_x_oauth2", build_file_proto_mode = "disable_global", importpath = "golang.org/x/oauth2", - sum = "h1:8tDJ3aechhddbdPAxpycgXHJRMLpk/Ab+aa4OgdN5/g=", - version = "v0.0.0-20220608161450-d0670ef3b1eb", + sum = "h1:OSnWWcOd/CtWQC2cYSBgbTSJv3ciqd8r54ySIW2y3RE=", + version = "v0.0.0-20220411215720-9780585627b5", ) go_repository( name = "org_golang_x_sync", @@ -4412,8 +4412,8 @@ def go_deps(): name = "org_golang_x_term", build_file_proto_mode = "disable_global", importpath = "golang.org/x/term", - sum = "h1:EH1Deb8WZJ0xc0WK//leUHXcX9aLE5SymusoTmMZye8=", - version = "v0.0.0-20220411215600-e5f449aeb171", + sum = "h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY=", + version = "v0.0.0-20210927222741-03fcf44c2211", ) go_repository( name = "org_golang_x_text", @@ -4440,8 +4440,8 @@ def go_deps(): name = "org_golang_x_xerrors", build_file_proto_mode = "disable_global", importpath = "golang.org/x/xerrors", - sum = "h1:uF6paiQQebLeSXkrTqHqz0MXhXXS1KgF41eUdBNvxK0=", - version = "v0.0.0-20220609144429-65e65417b02f", + sum = "h1:GGU+dLjvlC3qDwqYgL6UgRmHXhOOgns0bZu2Ty5mm6U=", + version = "v0.0.0-20220411194840-2f41105eb62f", ) go_repository( name = "org_gonum_v1_gonum", From 450bdba1febf5f7112027be871c83d5fb0c71fc1 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 19:33:56 +0800 Subject: [PATCH 12/16] make lint happy --- meta/autoid/autoid_service.go | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index 35eabf0510dd7..2130ac5948669 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -89,7 +89,7 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien // The returned range is (min, max]: // case increment=1 & offset=1: you can derive the ids like min+1, min+2... max. // case increment=x & offset=y: you firstly need to seek to firstID by `SeekToFirstAutoIDXXX`, then derive the IDs like firstID, firstID + increment * 2... in the caller. -func (sp *singlePointAlloc) Alloc(ctx context.Context, n uint64, increment, offset int64) (int64, int64, error) { +func (sp *singlePointAlloc) Alloc(ctx context.Context, n uint64, increment, offset int64) (min int64, max int64, _ error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("autoid.Alloc", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -127,16 +127,14 @@ retry: du := time.Since(start) metrics.AutoIDReqDuration.Observe(du.Seconds()) - if err == nil { - sp.lastAllocated = resp.Min - } + sp.lastAllocated = resp.Min return resp.Min, resp.Max, err } // AllocSeqCache allocs sequence batch value cached in table level(rather than in alloc), the returned range covering // the size of sequence cache with it's increment. The returned round indicates the sequence cycle times if it is with // cycle option. -func (sp *singlePointAlloc) AllocSeqCache() (min int64, max int64, round int64, err error) { +func (*singlePointAlloc) AllocSeqCache() (int64, int64, int64, error) { return 0, 0, 0, errors.New("AllocSeqCache not implemented") } @@ -150,10 +148,10 @@ func (sp *singlePointAlloc) Rebase(ctx context.Context, newBase int64, allocIDs ctx = opentracing.ContextWithSpan(ctx, span1) } - return sp.rebase(ctx, newBase, allocIDs, false) + return sp.rebase(ctx, newBase, false) } -func (sp *singlePointAlloc) rebase(ctx context.Context, newBase int64, allocIDs bool, force bool) error { +func (sp *singlePointAlloc) rebase(ctx context.Context, newBase int64, force bool) error { cli, err := sp.GetClient(ctx) if err != nil { return errors.Trace(err) @@ -176,11 +174,11 @@ func (sp *singlePointAlloc) ForceRebase(newBase int64) error { if newBase == -1 { return ErrAutoincReadFailed.GenWithStack("Cannot force rebase the next global ID to '0'") } - return sp.rebase(context.Background(), newBase, false, true) + return sp.rebase(context.Background(), newBase, true) } // RebaseSeq rebases the sequence value in number axis with tableID and the new base value. -func (sp *singlePointAlloc) RebaseSeq(newBase int64) (int64, bool, error) { +func (*singlePointAlloc) RebaseSeq(_ int64) (int64, bool, error) { return 0, false, errors.New("RebaseSeq not implemented") } @@ -202,6 +200,6 @@ func (sp *singlePointAlloc) NextGlobalAutoID() (int64, error) { return max + 1, err } -func (sp *singlePointAlloc) GetType() AllocatorType { +func (*singlePointAlloc) GetType() AllocatorType { return RowIDAllocType } From 23a26f1b599dd5db30578d7072641111acc186fb Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 19:58:00 +0800 Subject: [PATCH 13/16] fix lint --- meta/autoid/autoid_service.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index 2130ac5948669..60144805f934d 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -134,14 +134,14 @@ retry: // AllocSeqCache allocs sequence batch value cached in table level(rather than in alloc), the returned range covering // the size of sequence cache with it's increment. The returned round indicates the sequence cycle times if it is with // cycle option. -func (*singlePointAlloc) AllocSeqCache() (int64, int64, int64, error) { +func (*singlePointAlloc) AllocSeqCache() (a int64, b int64, c int64, err error) { return 0, 0, 0, errors.New("AllocSeqCache not implemented") } // Rebase rebases the autoID base for table with tableID and the new base value. // If allocIDs is true, it will allocate some IDs and save to the cache. // If allocIDs is false, it will not allocate IDs. -func (sp *singlePointAlloc) Rebase(ctx context.Context, newBase int64, allocIDs bool) error { +func (sp *singlePointAlloc) Rebase(ctx context.Context, newBase int64, _ bool) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("autoid.Rebase", opentracing.ChildOf(span.Context())) defer span1.Finish() From b5e77c0c918591b9e58dabb49ff13a0584099798 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 21:42:04 +0800 Subject: [PATCH 14/16] fix the GetClient() grpc conn leak --- meta/autoid/autoid.go | 4 +++- meta/autoid/autoid_service.go | 42 ++++++++++++++++++++++++++++++----- 2 files changed, 39 insertions(+), 7 deletions(-) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index ebb43256faa09..795eab2f29df7 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -559,7 +559,8 @@ func newSinglePointAlloc(store kv.Storage, dbID, tblID int64, isUnsigned bool) * TLS: ebd.TLSConfig(), }) if err != nil { - panic(err) + logutil.BgLogger().Error("[autoid client] fail to connect etcd, fallback to default", zap.Error(err)) + return nil } spa.clientDiscover = clientDiscover{etcdCli: etcdCli} } else { @@ -567,6 +568,7 @@ func newSinglePointAlloc(store kv.Storage, dbID, tblID int64, isUnsigned bool) * spa.mu.AutoIDAllocClient = autoid.MockForTest(store) } + // mockAutoIDChange failpoint is not implemented in this allocator, so fallback to use the default one. failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { if val.(bool) { spa = nil diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index 60144805f934d..f1f5850d7abc7 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -24,7 +24,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/autoid" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/util/logutil" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" ) @@ -46,6 +48,9 @@ type clientDiscover struct { mu struct { sync.RWMutex autoid.AutoIDAllocClient + // Release the client conn to avoid resource leak! + // See https://github.com/grpc/grpc-go/issues/5321 + *grpc.ClientConn } } @@ -62,6 +67,11 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien } d.mu.RUnlock() + d.mu.Lock() + defer d.mu.Unlock() + if d.mu.AutoIDAllocClient != nil { + return d.mu.AutoIDAllocClient, nil + } resp, err := d.etcdCli.Get(ctx, autoIDLeaderPath, clientv3.WithFirstCreate()...) if err != nil { return nil, errors.Trace(err) @@ -79,6 +89,7 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien cli = autoid.NewAutoIDAllocClient(grpcConn) d.mu.Lock() d.mu.AutoIDAllocClient = cli + d.mu.ClientConn = grpcConn d.mu.Unlock() return cli, nil } @@ -117,9 +128,8 @@ retry: }) if err != nil { if strings.Contains(err.Error(), "rpc error") { - sp.mu.Lock() - sp.mu.AutoIDAllocClient = nil - sp.mu.Unlock() + time.Sleep(200 * time.Millisecond) + sp.reConnect() goto retry } return 0, 0, errors.Trace(err) @@ -131,6 +141,20 @@ retry: return resp.Min, resp.Max, err } +func (sp *singlePointAlloc) reConnect() { + var grpcConn *grpc.ClientConn + sp.mu.Lock() + grpcConn = sp.mu.ClientConn + sp.mu.AutoIDAllocClient = nil + sp.mu.ClientConn = nil + sp.mu.Unlock() + // Close grpc.ClientConn to release resource. + if grpcConn != nil { + err := grpcConn.Close() + logutil.BgLogger().Info("[autoid client] AllocAutoID grpc error, reconnect", zap.Error(err)) + } +} + // AllocSeqCache allocs sequence batch value cached in table level(rather than in alloc), the returned range covering // the size of sequence cache with it's increment. The returned round indicates the sequence cycle times if it is with // cycle option. @@ -152,6 +176,7 @@ func (sp *singlePointAlloc) Rebase(ctx context.Context, newBase int64, _ bool) e } func (sp *singlePointAlloc) rebase(ctx context.Context, newBase int64, force bool) error { +retry: cli, err := sp.GetClient(ctx) if err != nil { return errors.Trace(err) @@ -163,9 +188,15 @@ func (sp *singlePointAlloc) rebase(ctx context.Context, newBase int64, force boo Force: force, IsUnsigned: sp.isUnsigned, }) - if err == nil { - sp.lastAllocated = newBase + if err != nil { + if strings.Contains(err.Error(), "rpc error") { + time.Sleep(200 * time.Millisecond) + sp.reConnect() + goto retry + } + return errors.Trace(err) } + sp.lastAllocated = newBase return err } @@ -195,7 +226,6 @@ func (sp *singlePointAlloc) End() int64 { // NextGlobalAutoID returns the next global autoID. // Used by 'show create table', 'alter table auto_increment = xxx' func (sp *singlePointAlloc) NextGlobalAutoID() (int64, error) { - // return sp.lastAllocated + 1, nil _, max, err := sp.Alloc(context.Background(), 0, 1, 1) return max + 1, err } From a62d96ca00e7e4c480cf3cf1341650a6a2ed3d2b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 21:56:13 +0800 Subject: [PATCH 15/16] tiny clean up --- meta/autoid/autoid_service.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index f1f5850d7abc7..fc42ca040af37 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -72,6 +72,7 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien if d.mu.AutoIDAllocClient != nil { return d.mu.AutoIDAllocClient, nil } + resp, err := d.etcdCli.Get(ctx, autoIDLeaderPath, clientv3.WithFirstCreate()...) if err != nil { return nil, errors.Trace(err) @@ -87,10 +88,8 @@ func (d *clientDiscover) GetClient(ctx context.Context) (autoid.AutoIDAllocClien return nil, errors.Trace(err) } cli = autoid.NewAutoIDAllocClient(grpcConn) - d.mu.Lock() d.mu.AutoIDAllocClient = cli d.mu.ClientConn = grpcConn - d.mu.Unlock() return cli, nil } @@ -129,7 +128,7 @@ retry: if err != nil { if strings.Contains(err.Error(), "rpc error") { time.Sleep(200 * time.Millisecond) - sp.reConnect() + sp.resetConn() goto retry } return 0, 0, errors.Trace(err) @@ -141,7 +140,7 @@ retry: return resp.Min, resp.Max, err } -func (sp *singlePointAlloc) reConnect() { +func (sp *singlePointAlloc) resetConn() { var grpcConn *grpc.ClientConn sp.mu.Lock() grpcConn = sp.mu.ClientConn @@ -191,7 +190,7 @@ retry: if err != nil { if strings.Contains(err.Error(), "rpc error") { time.Sleep(200 * time.Millisecond) - sp.reConnect() + sp.resetConn() goto retry } return errors.Trace(err) From 073f83995ca9173892df28542baf6a72846d1e47 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 3 Nov 2022 23:26:12 +0800 Subject: [PATCH 16/16] address comment --- meta/autoid/autoid.go | 1 - meta/autoid/autoid_service.go | 6 ++++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 795eab2f29df7..def3245bb2da3 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -468,7 +468,6 @@ func (alloc *allocator) ForceRebase(requiredBase int64) error { if requiredBase == -1 { return ErrAutoincReadFailed.GenWithStack("Cannot force rebase the next global ID to '0'") } - alloc.mu.Lock() defer alloc.mu.Unlock() startTime := time.Now() diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index fc42ca040af37..6133dfdfc3cb2 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -127,7 +127,7 @@ retry: }) if err != nil { if strings.Contains(err.Error(), "rpc error") { - time.Sleep(200 * time.Millisecond) + time.Sleep(backoffDuration) sp.resetConn() goto retry } @@ -140,6 +140,8 @@ retry: return resp.Min, resp.Max, err } +const backoffDuration = 200 * time.Millisecond + func (sp *singlePointAlloc) resetConn() { var grpcConn *grpc.ClientConn sp.mu.Lock() @@ -189,7 +191,7 @@ retry: }) if err != nil { if strings.Contains(err.Error(), "rpc error") { - time.Sleep(200 * time.Millisecond) + time.Sleep(backoffDuration) sp.resetConn() goto retry }