diff --git a/Makefile b/Makefile index 6e00960f7dbda..3254d88aab401 100644 --- a/Makefile +++ b/Makefile @@ -410,20 +410,20 @@ bazel_test: failpoint-enable bazel_ci_prepare bazel_coverage_test: failpoint-enable bazel_ci_prepare - bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) \ - --build_event_json_file=bazel_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest \ + bazel $(BAZEL_GLOBAL_CONFIG) --nohome_rc coverage $(BAZEL_CMD_CONFIG) --local_ram_resources=30720 --jobs=25 \ + --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest \ -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... - bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) \ - --build_event_json_file=bazel_1.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest,distributereorg \ + bazel $(BAZEL_GLOBAL_CONFIG) --nohome_rc coverage $(BAZEL_CMD_CONFIG) --local_ram_resources=30720 --jobs=25 \ + --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=deadlock,intest,distributereorg \ -- //... -//cmd/... -//tests/graceshutdown/... \ -//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/... bazel_build: bazel_ci_prepare mkdir -p bin - bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \ + bazel $(BAZEL_GLOBAL_CONFIG) --nohome_rc build $(BAZEL_CMD_CONFIG) --local_ram_resources=61440 --jobs=25 \ //... --//build:with_nogo_flag=true - bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \ + bazel $(BAZEL_GLOBAL_CONFIG) --nohome_rc build $(BAZEL_CMD_CONFIG) \ //cmd/importer:importer //tidb-server:tidb-server //tidb-server:tidb-server-check --//build:with_nogo_flag=true cp bazel-out/k8-fastbuild/bin/tidb-server/tidb-server_/tidb-server ./bin cp bazel-out/k8-fastbuild/bin/cmd/importer/importer_/importer ./bin diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 2d0ae75408ebb..c818c479e1fca 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/failpoint" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/backup" @@ -1736,6 +1737,15 @@ func (rc *Client) PreCheckTableTiFlashReplica( tables []*metautil.Table, recorder *tiflashrec.TiFlashRecorder, ) error { + // For TiDB 6.6, we do not support recover TiFlash replica while enabling API V2. + // TODO(iosmanthus): remove this after TiFlash support API V2. + if rc.GetDomain().Store().GetCodec().GetAPIVersion() == kvrpcpb.APIVersion_V2 { + log.Warn("TiFlash does not support API V2, reset replica count to 0") + for _, table := range tables { + table.Info.TiFlashReplica = nil + } + return nil + } tiFlashStoreCount, err := rc.getTiFlashNodeCount(ctx) if err != nil { return err diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 2e95e345c534d..080f2ccbb1584 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -226,6 +226,7 @@ go_test( "//errno", "//executor", "//infoschema", + "//keyspace", "//kv", "//meta", "//meta/autoid", diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index c9497f2381f52..4d5300d600470 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/gcutil" @@ -273,7 +274,7 @@ PARTITION BY RANGE (c) ( func TestFlashbackTable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -331,7 +332,7 @@ PARTITION BY RANGE (c) ( func TestDropTable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -384,7 +385,7 @@ PARTITION BY RANGE (c) ( func TestCreateWithSameName(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -448,7 +449,7 @@ PARTITION BY RANGE (c) ( func TestPartition(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -508,7 +509,7 @@ PARTITION BY RANGE (c) ( func TestDropSchema(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -534,7 +535,7 @@ PARTITION BY RANGE (c) ( func TestDefaultKeyword(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/ddl/main_test.go b/ddl/main_test.go index 84a713dc59bb3..b1bbd8516fa94 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/testkit/testsetup" "github.com/tikv/client-go/v2/tikv" @@ -54,7 +55,7 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) if err != nil { _, _ = fmt.Fprintf(os.Stderr, "ddl: infosync.GlobalInfoSyncerInit: %v\n", err) os.Exit(1) diff --git a/domain/BUILD.bazel b/domain/BUILD.bazel index ccbf75dd48ee6..ccb70230b9829 100644 --- a/domain/BUILD.bazel +++ b/domain/BUILD.bazel @@ -113,6 +113,7 @@ go_test( "//ddl", "//domain/infosync", "//errno", + "//keyspace", "//kv", "//metrics", "//parser/ast", diff --git a/domain/db_test.go b/domain/db_test.go index 428b63b6de05b..02f716a27b3a8 100644 --- a/domain/db_test.go +++ b/domain/db_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" @@ -73,7 +74,7 @@ func TestNormalSessionPool(t *testing.T) { domain, err := session.BootstrapSession(store) require.NoError(t, err) defer domain.Close() - info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, true) + info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) require.NoError(t, err1) conf := config.GetGlobalConfig() conf.Socket = "" @@ -107,7 +108,7 @@ func TestAbnormalSessionPool(t *testing.T) { domain, err := session.BootstrapSession(store) require.NoError(t, err) defer domain.Close() - info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, true) + info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) require.NoError(t, err1) conf := config.GetGlobalConfig() conf.Socket = "" diff --git a/domain/domain.go b/domain/domain.go index 07b69fb365f4f..b4a7a2770afd3 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1069,7 +1069,9 @@ func (do *Domain) Init( // step 1: prepare the info/schema syncer which domain reload needed. pdCli := do.GetPDClient() skipRegisterToDashboard := config.GetGlobalConfig().SkipRegisterToDashboard - do.info, err = infosync.GlobalInfoSyncerInit(ctx, do.ddl.GetID(), do.ServerID, do.etcdClient, do.unprefixedEtcdCli, pdCli, skipRegisterToDashboard) + do.info, err = infosync.GlobalInfoSyncerInit(ctx, do.ddl.GetID(), do.ServerID, + do.etcdClient, do.unprefixedEtcdCli, pdCli, do.Store().GetCodec(), + skipRegisterToDashboard) if err != nil { return err } diff --git a/domain/infosync/BUILD.bazel b/domain/infosync/BUILD.bazel index 0952dfc300490..f050873ee128f 100644 --- a/domain/infosync/BUILD.bazel +++ b/domain/infosync/BUILD.bazel @@ -41,10 +41,12 @@ go_library( "@com_github_gorilla_mux//:mux", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_kvproto//pkg/resource_manager", "@com_github_pingcap_log//:log", "@com_github_tikv_client_go_v2//oracle", + "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_pd_client//:client", "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_client_v3//concurrency", @@ -62,6 +64,7 @@ go_test( deps = [ "//ddl/placement", "//ddl/util", + "//keyspace", "//parser/model", "//testkit/testsetup", "//util", diff --git a/domain/infosync/info.go b/domain/infosync/info.go index b45216daa8a9f..1e82ce308ac9b 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/versioninfo" "github.com/tikv/client-go/v2/oracle" + "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/concurrency" @@ -192,6 +193,7 @@ func GlobalInfoSyncerInit( serverIDGetter func() uint64, etcdCli, unprefixedEtcdCli *clientv3.Client, pdCli pd.Client, + codec tikv.Codec, skipRegisterToDashBoard bool, ) (*InfoSyncer, error) { is := &InfoSyncer{ @@ -208,7 +210,7 @@ func GlobalInfoSyncerInit( is.labelRuleManager = initLabelRuleManager(etcdCli) is.placementManager = initPlacementManager(etcdCli) is.scheduleManager = initScheduleManager(etcdCli) - is.tiflashReplicaManager = initTiFlashReplicaManager(etcdCli) + is.tiflashReplicaManager = initTiFlashReplicaManager(etcdCli, codec) is.resourceGroupManager = initResourceGroupManager(pdCli) setGlobalInfoSyncer(is) return is, nil @@ -261,13 +263,13 @@ func initResourceGroupManager(pdCli pd.Client) pd.ResourceManagerClient { return pdCli } -func initTiFlashReplicaManager(etcdCli *clientv3.Client) TiFlashReplicaManager { +func initTiFlashReplicaManager(etcdCli *clientv3.Client, codec tikv.Codec) TiFlashReplicaManager { if etcdCli == nil { m := mockTiFlashReplicaManagerCtx{tiflashProgressCache: make(map[int64]float64)} return &m } logutil.BgLogger().Warn("init TiFlashReplicaManager", zap.Strings("pd addrs", etcdCli.Endpoints())) - return &TiFlashReplicaManagerCtx{etcdCli: etcdCli, tiflashProgressCache: make(map[int64]float64)} + return &TiFlashReplicaManagerCtx{etcdCli: etcdCli, tiflashProgressCache: make(map[int64]float64), codec: codec} } func initScheduleManager(etcdCli *clientv3.Client) ScheduleManager { diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index b7af5ed167837..ee97406eef01a 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit/testsetup" util2 "github.com/pingcap/tidb/util" @@ -67,7 +68,7 @@ func TestTopology(t *testing.T) { require.NoError(t, err) }() - info, err := GlobalInfoSyncerInit(ctx, currentID, func() uint64 { return 1 }, client, client, nil, false) + info, err := GlobalInfoSyncerInit(ctx, currentID, func() uint64 { return 1 }, client, client, nil, keyspace.CodecV1, false) require.NoError(t, err) err = info.newTopologySessionAndStoreServerInfo(ctx, util2.NewSessionDefaultRetryCnt) @@ -152,7 +153,7 @@ func (is *InfoSyncer) ttlKeyExists(ctx context.Context) (bool, error) { } func TestPutBundlesRetry(t *testing.T) { - _, err := GlobalInfoSyncerInit(context.TODO(), "test", func() uint64 { return 1 }, nil, nil, nil, false) + _, err := GlobalInfoSyncerInit(context.TODO(), "test", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, false) require.NoError(t, err) bundle, err := placement.NewBundleFromOptions(&model.PlacementSettings{PrimaryRegion: "r1", Regions: "r1,r2"}) @@ -216,7 +217,7 @@ func TestPutBundlesRetry(t *testing.T) { func TestTiFlashManager(t *testing.T) { ctx := context.Background() - _, err := GlobalInfoSyncerInit(ctx, "test", func() uint64 { return 1 }, nil, nil, nil, false) + _, err := GlobalInfoSyncerInit(ctx, "test", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, false) tiflash := NewMockTiFlash() SetMockTiFlash(tiflash) diff --git a/domain/infosync/tiflash_manager.go b/domain/infosync/tiflash_manager.go index d5cc46f95db95..6e47d881215d4 100644 --- a/domain/infosync/tiflash_manager.go +++ b/domain/infosync/tiflash_manager.go @@ -32,12 +32,15 @@ import ( "github.com/gorilla/mux" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" + "github.com/tikv/client-go/v2/tikv" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -77,6 +80,7 @@ type TiFlashReplicaManagerCtx struct { etcdCli *clientv3.Client sync.RWMutex // protect tiflashProgressCache tiflashProgressCache map[int64]float64 + codec tikv.Codec } // Close is called to close TiFlashReplicaManagerCtx. @@ -230,6 +234,11 @@ func (m *TiFlashReplicaManagerCtx) SetTiFlashGroupConfig(ctx context.Context) er // SetPlacementRule is a helper function to set placement rule. func (m *TiFlashReplicaManagerCtx) SetPlacementRule(ctx context.Context, rule placement.TiFlashRule) error { + // TiDB 6.6 doesn't support tiflash multi-tenancy yet. + // TODO(iosmanthus): remove this check after TiDB supports tiflash multi-tenancy. + if m.codec.GetAPIVersion() == kvrpcpb.APIVersion_V2 { + return errors.Trace(dbterror.ErrNotSupportedYet.GenWithStackByArgs("set TiFlash replica count while enabling API V2")) + } if err := m.SetTiFlashGroupConfig(ctx); err != nil { return err } diff --git a/server/BUILD.bazel b/server/BUILD.bazel index c0477b9248e5c..2f7de500099cf 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -162,6 +162,7 @@ go_test( "//expression", "//extension", "//infoschema", + "//keyspace", "//kv", "//meta", "//metrics", diff --git a/server/stat_test.go b/server/stat_test.go index 2cea08933c2c0..dfa2228467911 100644 --- a/server/stat_test.go +++ b/server/stat_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/keyspace" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/stretchr/testify/require" @@ -46,7 +47,7 @@ func TestUptime(t *testing.T) { }() require.NoError(t, err) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), true) + _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) require.NoError(t, err) tidbdrv := NewTiDBDriver(store) diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index 8dccd523fc887..7186245a79bda 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -82,6 +82,7 @@ go_test( data = glob(["testdata/**"]), embed = [":statistics"], flaky = True, + shard_count = 50, deps = [ "//config", "//domain",