diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel index 6f1a13742ca80..26eb992c89474 100644 --- a/autoid_service/BUILD.bazel +++ b/autoid_service/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "autoid_service", @@ -9,6 +9,7 @@ go_library( "//config", "//kv", "//meta", + "//meta/autoid", "//metrics", "//owner", "//parser/model", @@ -23,3 +24,18 @@ go_library( "@org_uber_go_zap//:zap", ], ) + +go_test( + name = "autoid_service_test", + srcs = ["autoid_test.go"], + embed = [":autoid_service"], + deps = [ + "//parser/model", + "//testkit", + "@com_github_pingcap_kvproto//pkg/autoid", + "@com_github_stretchr_testify//require", + "@io_etcd_go_etcd_tests_v3//integration", + "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//credentials/insecure", + ], +) diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index 1a4d2b426263e..aa6c487cb0b48 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" + autoid1 "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/parser/model" @@ -253,6 +254,7 @@ type Service struct { func New(selfAddr string, etcdAddr []string, store kv.Storage, tlsConfig *tls.Config) *Service { cfg := config.GetGlobalConfig() etcdLogCfg := zap.NewProductionConfig() + cli, err := clientv3.New(clientv3.Config{ LogConfig: &etcdLogCfg, Endpoints: etcdAddr, @@ -270,9 +272,12 @@ func New(selfAddr string, etcdAddr []string, store kv.Storage, tlsConfig *tls.Co if err != nil { panic(err) } + return newWithCli(selfAddr, cli, store) +} +func newWithCli(selfAddr string, cli *clientv3.Client, store kv.Storage) *Service { l := owner.NewOwnerManager(context.Background(), cli, "autoid", selfAddr, autoIDLeaderPath) - err = l.CampaignOwner() + err := l.CampaignOwner() if err != nil { panic(err) } @@ -299,7 +304,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 { +func MockForTest(store kv.Storage) autoid.AutoIDAllocClient { uuid := store.UUID() ret, ok := global[uuid] if !ok { @@ -515,3 +520,7 @@ func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoi } return &autoid.RebaseResponse{}, nil } + +func init() { + autoid1.MockForTest = MockForTest +} diff --git a/autoid_service/autoid_test.go b/autoid_service/autoid_test.go new file mode 100644 index 0000000000000..df2722309cf6e --- /dev/null +++ b/autoid_service/autoid_test.go @@ -0,0 +1,202 @@ +// 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" + "fmt" + "math" + "net" + "testing" + "time" + + "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" + "go.etcd.io/etcd/tests/v3/integration" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +type autoIDResp struct { + *autoid.AutoIDResponse + error + *testing.T +} + +func (resp autoIDResp) check(min, max int64) { + require.NoError(resp.T, resp.error) + require.Equal(resp.T, resp.AutoIDResponse, &autoid.AutoIDResponse{Min: min, Max: max}) +} + +func (resp autoIDResp) checkErrmsg() { + require.NoError(resp.T, resp.error) + require.True(resp.T, len(resp.GetErrmsg()) > 0) +} + +type rebaseResp struct { + *autoid.RebaseResponse + error + *testing.T +} + +func (resp rebaseResp) check(msg string) { + require.NoError(resp.T, resp.error) + require.Equal(resp.T, string(resp.RebaseResponse.GetErrmsg()), msg) +} + +func TestAPI(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + cli := MockForTest(store) + tk.MustExec("use test") + tk.MustExec("create table t (id int key auto_increment);") + is := dom.InfoSchema() + dbInfo, ok := is.SchemaByName(model.NewCIStr("test")) + require.True(t, ok) + + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tbInfo := tbl.Meta() + + ctx := context.Background() + checkCurrValue := func(t *testing.T, cli autoid.AutoIDAllocClient, min, max int64) { + req := &autoid.AutoIDRequest{DbID: dbInfo.ID, TblID: tbInfo.ID, N: 0} + resp, err := cli.AllocAutoID(ctx, req) + require.NoError(t, err) + require.Equal(t, resp, &autoid.AutoIDResponse{Min: min, Max: max}) + } + autoIDRequest := func(t *testing.T, cli autoid.AutoIDAllocClient, unsigned bool, n uint64, more ...int64) autoIDResp { + increment := int64(1) + offset := int64(1) + if len(more) >= 1 { + increment = more[0] + } + if len(more) >= 2 { + offset = more[1] + } + req := &autoid.AutoIDRequest{DbID: dbInfo.ID, TblID: tbInfo.ID, IsUnsigned: unsigned, N: n, Increment: increment, Offset: offset} + resp, err := cli.AllocAutoID(ctx, req) + return autoIDResp{resp, err, t} + } + rebaseRequest := func(t *testing.T, cli autoid.AutoIDAllocClient, unsigned bool, n int64, force ...struct{}) rebaseResp { + req := &autoid.RebaseRequest{ + DbID: dbInfo.ID, + TblID: tbInfo.ID, + Base: n, + IsUnsigned: unsigned, + Force: len(force) > 0, + } + resp, err := cli.Rebase(ctx, req) + return rebaseResp{resp, err, t} + } + var force = struct{}{} + + // basic auto id operation + autoIDRequest(t, cli, false, 1).check(0, 1) + autoIDRequest(t, cli, false, 10).check(1, 11) + checkCurrValue(t, cli, 11, 11) + autoIDRequest(t, cli, false, 128).check(11, 139) + autoIDRequest(t, cli, false, 1, 10, 5).check(139, 145) + + // basic rebase operation + rebaseRequest(t, cli, false, 666).check("") + autoIDRequest(t, cli, false, 1).check(666, 667) + + rebaseRequest(t, cli, false, 6666).check("") + autoIDRequest(t, cli, false, 1).check(6666, 6667) + + // rebase will not decrease the value without 'force' + rebaseRequest(t, cli, false, 44).check("") + checkCurrValue(t, cli, 6667, 6667) + rebaseRequest(t, cli, false, 44, force).check("") + checkCurrValue(t, cli, 44, 44) + + // max increase 1 + rebaseRequest(t, cli, false, math.MaxInt64, force).check("") + checkCurrValue(t, cli, math.MaxInt64, math.MaxInt64) + autoIDRequest(t, cli, false, 1).checkErrmsg() + + rebaseRequest(t, cli, true, 0, force).check("") + checkCurrValue(t, cli, 0, 0) + autoIDRequest(t, cli, true, 1).check(0, 1) + autoIDRequest(t, cli, true, 10).check(1, 11) + autoIDRequest(t, cli, true, 128).check(11, 139) + autoIDRequest(t, cli, true, 1, 10, 5).check(139, 145) + + // max increase 1 + rebaseRequest(t, cli, true, math.MaxInt64).check("") + checkCurrValue(t, cli, math.MaxInt64, math.MaxInt64) + autoIDRequest(t, cli, true, 1).check(math.MaxInt64, math.MinInt64) + autoIDRequest(t, cli, true, 1).check(math.MinInt64, math.MinInt64+1) + + rebaseRequest(t, cli, true, -1).check("") + checkCurrValue(t, cli, -1, -1) + autoIDRequest(t, cli, true, 1).check(-1, 0) +} + +func TestGRPC(t *testing.T) { + integration.BeforeTestExternal(t) + store := testkit.CreateMockStore(t) + cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer cluster.Terminate(t) + etcdCli := cluster.RandClient() + + var addr string + var listener net.Listener + for port := 10080; ; port++ { + var err error + addr = fmt.Sprintf("127.0.0.1:%d", port) + listener, err = net.Listen("tcp", addr) + if err == nil { + break + } + } + defer listener.Close() + + service := newWithCli(addr, etcdCli, store) + defer service.Close() + + var i int + for !service.leaderShip.IsOwner() { + time.Sleep(100 * time.Millisecond) + i++ + if i >= 20 { + break + } + } + require.Less(t, i, 20) + + grpcServer := grpc.NewServer() + autoid.RegisterAutoIDAllocServer(grpcServer, service) + go func() { + grpcServer.Serve(listener) + }() + defer grpcServer.Stop() + + grpcConn, err := grpc.Dial("127.0.0.1:10080", grpc.WithTransportCredentials(insecure.NewCredentials())) + require.NoError(t, err) + cli := autoid.NewAutoIDAllocClient(grpcConn) + _, err = cli.AllocAutoID(context.Background(), &autoid.AutoIDRequest{ + DbID: 0, + TblID: 0, + N: 1, + Increment: 1, + Offset: 1, + IsUnsigned: false, + }) + require.NoError(t, err) +} diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index dc179250ad4bd..dce469fe15321 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -202,6 +202,7 @@ go_test( flaky = True, shard_count = 50, deps = [ + "//autoid_service", "//config", "//ddl/ingest", "//ddl/placement", diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index cc9cc657fdc6f..1d482f8cecada 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -26,6 +26,7 @@ import ( "time" "github.com/pingcap/errors" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/schematracker" diff --git a/executor/autoidtest/BUILD.bazel b/executor/autoidtest/BUILD.bazel index 0f5bf6c434f91..cd04b266fa2e3 100644 --- a/executor/autoidtest/BUILD.bazel +++ b/executor/autoidtest/BUILD.bazel @@ -9,6 +9,7 @@ go_test( flaky = True, race = "on", deps = [ + "//autoid_service", "//config", "//ddl/testutil", "//meta/autoid", diff --git a/executor/autoidtest/autoid_test.go b/executor/autoidtest/autoid_test.go index 7823a7488bf98..eb8cc3f874159 100644 --- a/executor/autoidtest/autoid_test.go +++ b/executor/autoidtest/autoid_test.go @@ -22,6 +22,7 @@ import ( "testing" "github.com/pingcap/failpoint" + _ "github.com/pingcap/tidb/autoid_service" ddltestutil "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" diff --git a/executor/issuetest/BUILD.bazel b/executor/issuetest/BUILD.bazel index 77bfaf7f11290..8d930738d2b7c 100644 --- a/executor/issuetest/BUILD.bazel +++ b/executor/issuetest/BUILD.bazel @@ -9,6 +9,7 @@ go_test( flaky = True, shard_count = 50, deps = [ + "//autoid_service", "//config", "//kv", "//meta/autoid", diff --git a/executor/issuetest/executor_issue_test.go b/executor/issuetest/executor_issue_test.go index 9e28feca1530f..8dcbf251cdf89 100644 --- a/executor/issuetest/executor_issue_test.go +++ b/executor/issuetest/executor_issue_test.go @@ -22,6 +22,7 @@ import ( "testing" "github.com/pingcap/failpoint" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/auth" diff --git a/executor/showtest/BUILD.bazel b/executor/showtest/BUILD.bazel index 807e00c8e88ec..1882c92e0627d 100644 --- a/executor/showtest/BUILD.bazel +++ b/executor/showtest/BUILD.bazel @@ -11,6 +11,7 @@ go_test( race = "on", shard_count = 45, deps = [ + "//autoid_service", "//config", "//executor", "//infoschema", diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index 0573de30137f6..c327eb474e6d6 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/pingcap/failpoint" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/auth" diff --git a/meta/autoid/BUILD.bazel b/meta/autoid/BUILD.bazel index 50e53258f305b..b67f7f7c223c7 100644 --- a/meta/autoid/BUILD.bazel +++ b/meta/autoid/BUILD.bazel @@ -11,7 +11,6 @@ go_library( importpath = "github.com/pingcap/tidb/meta/autoid", visibility = ["//visibility:public"], deps = [ - "//autoid_service", "//config", "//errno", "//kv", diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index aba2ad565b617..1f5ffeb2fd094 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -26,7 +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/kvproto/pkg/autoid" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -558,6 +558,11 @@ func NextStep(curStep int64, consumeDur time.Duration) int64 { return res } +// MockForTest is exported for testing. +// The actual implementation is in github.com/pingcap/tidb/autoid_service because of the +// package circle depending issue. +var MockForTest func(kv.Storage) autoid.AutoIDAllocClient + func newSinglePointAlloc(store kv.Storage, dbID, tblID int64, isUnsigned bool) *singlePointAlloc { ebd, ok := store.(kv.EtcdBackend) if !ok { @@ -587,7 +592,7 @@ func newSinglePointAlloc(store kv.Storage, dbID, tblID int64, isUnsigned bool) * spa.clientDiscover = clientDiscover{etcdCli: etcdCli} } else { spa.clientDiscover = clientDiscover{} - spa.mu.AutoIDAllocClient = autoid.MockForTest(store) + spa.mu.AutoIDAllocClient = MockForTest(store) } // mockAutoIDChange failpoint is not implemented in this allocator, so fallback to use the default one. diff --git a/session/BUILD.bazel b/session/BUILD.bazel index dc3106abdfe63..63118a3ea701a 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -128,6 +128,7 @@ go_test( race = "on", shard_count = 50, deps = [ + "//autoid_service", "//bindinfo", "//config", "//ddl", diff --git a/session/bench_test.go b/session/bench_test.go index ece43c39cdc77..04c86b9227f8d 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/pingcap/log" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" diff --git a/sessionctx/binloginfo/BUILD.bazel b/sessionctx/binloginfo/BUILD.bazel index 6d5a600b9e68c..7a843495273ea 100644 --- a/sessionctx/binloginfo/BUILD.bazel +++ b/sessionctx/binloginfo/BUILD.bazel @@ -33,6 +33,7 @@ go_test( embed = [":binloginfo"], flaky = True, deps = [ + "//autoid_service", "//ddl", "//domain", "//kv", diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 3c777a9436234..28235b5184b68 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" diff --git a/telemetry/BUILD.bazel b/telemetry/BUILD.bazel index df7de986555a6..1f032aa3f237a 100644 --- a/telemetry/BUILD.bazel +++ b/telemetry/BUILD.bazel @@ -61,6 +61,7 @@ go_test( embed = [":telemetry"], flaky = True, deps = [ + "//autoid_service", "//config", "//ddl", "//domain", diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index cb3272d110b29..c303c53f3006b 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -18,6 +18,7 @@ import ( "fmt" "testing" + _ "github.com/pingcap/tidb/autoid_service" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/sessionctx/variable"