diff --git a/DEPS.bzl b/DEPS.bzl index 2b702f625b5f9..e8e9e771f96cc 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -7180,13 +7180,13 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sha256 = "bb5f9eb136d207f214c1eef12b89774edfa55e0b6721c66c7f83b1dfd6c11855", - strip_prefix = "github.com/tikv/client-go/v2@v2.0.8-0.20240604045705-156cebc2defa", + sha256 = "de925cd3715472de4dd5ca829e4bb6b0495cc0574af1c3ad2ae97dbf23b59786", + strip_prefix = "github.com/tikv/client-go/v2@v2.0.8-0.20240614064455-ac8fa1d73a0c", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240604045705-156cebc2defa.zip", - "http://ats.apps.svc/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240604045705-156cebc2defa.zip", - "https://cache.hawkingrei.com/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240604045705-156cebc2defa.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240604045705-156cebc2defa.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240614064455-ac8fa1d73a0c.zip", + "http://ats.apps.svc/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240614064455-ac8fa1d73a0c.zip", + "https://cache.hawkingrei.com/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240614064455-ac8fa1d73a0c.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20240614064455-ac8fa1d73a0c.zip", ], ) go_repository( diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go index d110abbc49dcc..31f20cd8af433 100644 --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -192,6 +192,15 @@ func NewPdController( }, nil } +func NewPdControllerWithPDClient(pdClient pd.Client, pdHTTPCli pdhttp.Client, v *semver.Version) *PdController { + return &PdController{ + pdClient: pdClient, + pdHTTPCli: pdHTTPCli, + version: v, + schedulerPauseCh: make(chan struct{}, 1), + } +} + func parseVersion(versionStr string) *semver.Version { // we need trim space or semver will parse failed v := strings.TrimSpace(versionStr) diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 5823273eaea88..41bf3989a7079 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -36,14 +36,26 @@ go_library( go_test( name = "restore_test", timeout = "short", - srcs = ["misc_test.go"], + srcs = [ + "import_mode_switcher_test.go", + "misc_test.go", + ], flaky = True, race = "off", + shard_count = 6, deps = [ ":restore", + "//br/pkg/conn", "//br/pkg/mock", + "//br/pkg/pdutil", + "//br/pkg/utiltest", "//pkg/infoschema", "//pkg/parser/model", + "@com_github_coreos_go_semver//semver", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/import_sstpb", + "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_stretchr_testify//require", + "@org_golang_google_grpc//:grpc", ], ) diff --git a/br/pkg/restore/import_mode_switcher_test.go b/br/pkg/restore/import_mode_switcher_test.go new file mode 100644 index 0000000000000..75d08c11a9033 --- /dev/null +++ b/br/pkg/restore/import_mode_switcher_test.go @@ -0,0 +1,145 @@ +// Copyright 2024 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 restore_test + +import ( + "context" + "fmt" + "net" + "sync" + "testing" + "time" + + "github.com/coreos/go-semver/semver" + "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/conn" + "github.com/pingcap/tidb/br/pkg/pdutil" + "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" +) + +type mockImportServer struct { + import_sstpb.ImportSSTServer + + count int + ch chan struct{} +} + +func (s *mockImportServer) SwitchMode(_ context.Context, req *import_sstpb.SwitchModeRequest) (*import_sstpb.SwitchModeResponse, error) { + s.count -= 1 + if s.count == 0 { + s.ch <- struct{}{} + } + return &import_sstpb.SwitchModeResponse{}, nil +} + +func TestRestorePreWork(t *testing.T) { + ctx := context.Background() + var port int + var lis net.Listener + var err error + for port = 0; port < 1000; port += 1 { + addr := fmt.Sprintf(":%d", 51111+port) + lis, err = net.Listen("tcp", addr) + if err == nil { + break + } + t.Log(err) + } + + s := grpc.NewServer() + ch := make(chan struct{}) + import_sstpb.RegisterImportSSTServer(s, &mockImportServer{count: 3, ch: ch}) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := s.Serve(lis) + require.NoError(t, err) + }() + + pdClient := utiltest.NewFakePDClient([]*metapb.Store{ + { + Id: 1, + Address: fmt.Sprintf(":%d", 51111+port), + }, + }, false, nil) + pdHTTPCli := utiltest.NewFakePDHTTPClient() + mgr := &conn.Mgr{ + PdController: pdutil.NewPdControllerWithPDClient( + pdClient, pdHTTPCli, &semver.Version{Major: 4, Minor: 0, Patch: 9}), + } + mgr.PdController.SchedulerPauseTTL = 3 * time.Second + switcher := restore.NewImportModeSwitcher(pdClient, time.Millisecond*200, nil) + undo, cfg, err := restore.RestorePreWork(ctx, mgr, switcher, false, true) + require.NoError(t, err) + // check the cfg + { + require.Equal(t, len(pdutil.Schedulers), len(cfg.Schedulers)) + for _, key := range cfg.Schedulers { + _, ok := pdutil.Schedulers[key] + require.True(t, ok) + } + require.Equal(t, len(utiltest.ExistPDCfgGeneratorBefore), len(cfg.ScheduleCfg)) + for key, value := range cfg.ScheduleCfg { + expectValue, ok := utiltest.ExistPDCfgGeneratorBefore[key] + require.True(t, ok) + require.Equal(t, expectValue, value) + } + cfgs, err := pdHTTPCli.GetConfig(context.TODO()) + require.NoError(t, err) + require.Equal(t, len(utiltest.ExpectPDCfgGeneratorsResult), len(cfg.ScheduleCfg)) + for key, value := range cfgs { + expectValue, ok := utiltest.ExpectPDCfgGeneratorsResult[key[len("schedule."):]] + require.True(t, ok) + require.Equal(t, expectValue, value) + } + delaySchedulers := pdHTTPCli.GetDelaySchedulers() + require.Equal(t, len(pdutil.Schedulers), len(delaySchedulers)) + for delayScheduler := range delaySchedulers { + _, ok := pdutil.Schedulers[delayScheduler] + require.True(t, ok) + } + } + <-ch + restore.RestorePostWork(ctx, switcher, undo, false) + // check the cfg done + { + cfgs, err := pdHTTPCli.GetConfig(context.TODO()) + require.NoError(t, err) + require.Equal(t, len(utiltest.ExistPDCfgGeneratorBefore), len(cfg.ScheduleCfg)) + for key, value := range cfgs { + expectValue, ok := utiltest.ExistPDCfgGeneratorBefore[key[len("schedule."):]] + require.True(t, ok) + require.Equal(t, expectValue, value) + } + delaySchedulers := pdHTTPCli.GetDelaySchedulers() + require.Equal(t, 0, len(delaySchedulers)) + } + + s.Stop() + lis.Close() +} + +func TestRestorePreWorkOnline(t *testing.T) { + ctx := context.Background() + undo, _, err := restore.RestorePreWork(ctx, nil, nil, true, false) + require.NoError(t, err) + restore.RestorePostWork(ctx, nil, undo, true) +} diff --git a/br/pkg/restore/internal/import_client/BUILD.bazel b/br/pkg/restore/internal/import_client/BUILD.bazel index 572bf61357d04..bf5e81d192814 100644 --- a/br/pkg/restore/internal/import_client/BUILD.bazel +++ b/br/pkg/restore/internal/import_client/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 = "import_client", @@ -18,3 +18,22 @@ go_library( "@org_golang_google_grpc//status", ], ) + +go_test( + name = "import_client_test", + timeout = "short", + srcs = ["import_client_test.go"], + flaky = True, + deps = [ + ":import_client", + "//br/pkg/restore/split", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/errorpb", + "@com_github_pingcap_kvproto//pkg/import_sstpb", + "@com_github_pingcap_kvproto//pkg/kvrpcpb", + "@com_github_pingcap_kvproto//pkg/metapb", + "@com_github_stretchr_testify//require", + "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//keepalive", + ], +) diff --git a/br/pkg/restore/internal/import_client/import_client_test.go b/br/pkg/restore/internal/import_client/import_client_test.go new file mode 100644 index 0000000000000..5fb2d8479deac --- /dev/null +++ b/br/pkg/restore/internal/import_client/import_client_test.go @@ -0,0 +1,148 @@ +// Copyright 2024 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 importclient_test + +import ( + "context" + "fmt" + "net" + "sync" + "testing" + + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/errorpb" + "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + importclient "github.com/pingcap/tidb/br/pkg/restore/internal/import_client" + "github.com/pingcap/tidb/br/pkg/restore/split" + "github.com/stretchr/testify/require" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" +) + +type storeClient struct { + split.SplitClient + addr string +} + +func (sc *storeClient) GetStore(_ context.Context, _ uint64) (*metapb.Store, error) { + return &metapb.Store{ + Address: sc.addr, + }, nil +} + +type mockImportServer struct { + import_sstpb.ImportSSTServer + + ErrCount int +} + +func (s *mockImportServer) ClearFiles(_ context.Context, req *import_sstpb.ClearRequest) (*import_sstpb.ClearResponse, error) { + return &import_sstpb.ClearResponse{Error: &import_sstpb.Error{Message: req.Prefix}}, nil +} + +func (s *mockImportServer) Apply(_ context.Context, req *import_sstpb.ApplyRequest) (*import_sstpb.ApplyResponse, error) { + return &import_sstpb.ApplyResponse{Error: &import_sstpb.Error{Message: req.StorageCacheId}}, nil +} + +func (s *mockImportServer) Download(_ context.Context, req *import_sstpb.DownloadRequest) (*import_sstpb.DownloadResponse, error) { + return &import_sstpb.DownloadResponse{Error: &import_sstpb.Error{Message: req.Name}}, nil +} + +func (s *mockImportServer) SetDownloadSpeedLimit(_ context.Context, req *import_sstpb.SetDownloadSpeedLimitRequest) (*import_sstpb.SetDownloadSpeedLimitResponse, error) { + return &import_sstpb.SetDownloadSpeedLimitResponse{}, nil +} + +func (s *mockImportServer) MultiIngest(_ context.Context, req *import_sstpb.MultiIngestRequest) (*import_sstpb.IngestResponse, error) { + if s.ErrCount <= 0 { + return nil, errors.Errorf("test") + } + s.ErrCount -= 1 + if req.Context == nil { + return &import_sstpb.IngestResponse{}, nil + } + return &import_sstpb.IngestResponse{Error: &errorpb.Error{Message: req.Context.RequestSource}}, nil +} + +func TestImportClient(t *testing.T) { + ctx := context.Background() + var port int + var lis net.Listener + var err error + for port = 0; port < 1000; port += 1 { + addr := fmt.Sprintf(":%d", 51111+port) + lis, err = net.Listen("tcp", addr) + if err == nil { + break + } + t.Log(err) + } + + s := grpc.NewServer() + import_sstpb.RegisterImportSSTServer(s, &mockImportServer{ErrCount: 3}) + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + err := s.Serve(lis) + require.NoError(t, err) + }() + + client := importclient.NewImportClient(&storeClient{addr: fmt.Sprintf(":%d", 51111+port)}, nil, keepalive.ClientParameters{}) + + { + resp, err := client.ClearFiles(ctx, 1, &import_sstpb.ClearRequest{Prefix: "test"}) + require.NoError(t, err) + require.Equal(t, "test", resp.Error.Message) + } + + { + resp, err := client.ApplyKVFile(ctx, 1, &import_sstpb.ApplyRequest{StorageCacheId: "test"}) + require.NoError(t, err) + require.Equal(t, "test", resp.Error.Message) + } + + { + resp, err := client.DownloadSST(ctx, 1, &import_sstpb.DownloadRequest{Name: "test"}) + require.NoError(t, err) + require.Equal(t, "test", resp.Error.Message) + } + + { + _, err := client.SetDownloadSpeedLimit(ctx, 1, &import_sstpb.SetDownloadSpeedLimitRequest{SpeedLimit: 123}) + require.NoError(t, err) + } + + { + resp, err := client.MultiIngest(ctx, 1, &import_sstpb.MultiIngestRequest{Context: &kvrpcpb.Context{RequestSource: "test"}}) + require.NoError(t, err) + require.Equal(t, "test", resp.Error.Message) + } + + { + err := client.CheckMultiIngestSupport(ctx, []uint64{1}) + require.NoError(t, err) + err = client.CheckMultiIngestSupport(ctx, []uint64{3, 4, 5}) + require.Error(t, err) + } + + err = client.CloseGrpcClient() + require.NoError(t, err) + + s.Stop() + lis.Close() +} diff --git a/br/pkg/restore/internal/log_split/BUILD.bazel b/br/pkg/restore/internal/log_split/BUILD.bazel index 7b3cc6bdd626f..ffe8b4f5f1dea 100644 --- a/br/pkg/restore/internal/log_split/BUILD.bazel +++ b/br/pkg/restore/internal/log_split/BUILD.bazel @@ -32,19 +32,23 @@ go_test( name = "log_split_test", timeout = "short", srcs = [ + "export_test.go", "split_test.go", "sum_sorted_test.go", ], + embed = [":log_split"], flaky = True, - shard_count = 3, + shard_count = 4, deps = [ - ":log_split", "//br/pkg/restore/internal/utils", "//br/pkg/restore/split", "//br/pkg/restore/utils", "//br/pkg/utiltest", + "//pkg/kv", "//pkg/tablecodec", "//pkg/util/codec", + "@com_github_docker_go_units//:go-units", + "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/import_sstpb", "@com_github_stretchr_testify//require", ], diff --git a/br/pkg/restore/internal/log_split/export_test.go b/br/pkg/restore/internal/log_split/export_test.go new file mode 100644 index 0000000000000..bca3eada27bbb --- /dev/null +++ b/br/pkg/restore/internal/log_split/export_test.go @@ -0,0 +1,29 @@ +// Copyright 2024 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 logsplit + +import restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" + +func NewSplitHelperIteratorForTest(helper *SplitHelper, tableID int64, rule *restoreutils.RewriteRules) *splitHelperIterator { + return &splitHelperIterator{ + tableSplitters: []*rewriteSplitter{ + { + tableID: tableID, + rule: rule, + splitter: helper, + }, + }, + } +} diff --git a/br/pkg/restore/internal/log_split/split.go b/br/pkg/restore/internal/log_split/split.go index 021135c1e71b5..1c2319d513425 100644 --- a/br/pkg/restore/internal/log_split/split.go +++ b/br/pkg/restore/internal/log_split/split.go @@ -56,18 +56,6 @@ func (iter *splitHelperIterator) Traverse(fn func(v Valued, endKey []byte, rule } } -func NewSplitHelperIteratorForTest(helper *SplitHelper, tableID int64, rule *restoreutils.RewriteRules) *splitHelperIterator { - return &splitHelperIterator{ - tableSplitters: []*rewriteSplitter{ - { - tableID: tableID, - rule: rule, - splitter: helper, - }, - }, - } -} - type LogSplitHelper struct { tableSplitter map[int64]*SplitHelper rules map[int64]*restoreutils.RewriteRules @@ -358,7 +346,7 @@ func (helper *LogSplitHelper) Split(ctx context.Context) error { receiveNewRegions: for { select { - case <-ectx.Done(): + case <-ctx.Done(): return case newRegions, ok := <-helper.regionsCh: if !ok { diff --git a/br/pkg/restore/internal/log_split/split_test.go b/br/pkg/restore/internal/log_split/split_test.go index 99e30b78b22b8..578d9eefb447d 100644 --- a/br/pkg/restore/internal/log_split/split_test.go +++ b/br/pkg/restore/internal/log_split/split_test.go @@ -19,12 +19,15 @@ import ( "fmt" "testing" + "github.com/docker/go-units" + backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/import_sstpb" logsplit "github.com/pingcap/tidb/br/pkg/restore/internal/log_split" "github.com/pingcap/tidb/br/pkg/restore/internal/utils" "github.com/pingcap/tidb/br/pkg/restore/split" restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" @@ -154,3 +157,74 @@ func TestSplitPoint2(t *testing.T) { }) require.NoError(t, err) } + +func fakeFile(tableID, rowID int64, length uint64, num int64) *backuppb.DataFileInfo { + return &backuppb.DataFileInfo{ + StartKey: fakeRowKey(tableID, rowID), + EndKey: fakeRowKey(tableID, rowID+1), + TableId: tableID, + Length: length, + NumberOfEntries: num, + } +} + +func fakeRowKey(tableID, rowID int64) kv.Key { + return codec.EncodeBytes(nil, tablecodec.EncodeRecordKey(tablecodec.GenTableRecordPrefix(tableID), kv.IntHandle(rowID))) +} + +func TestLogSplitHelper(t *testing.T) { + ctx := context.Background() + rules := map[int64]*restoreutils.RewriteRules{ + 1: { + Data: []*import_sstpb.RewriteRule{ + { + OldKeyPrefix: tablecodec.GenTableRecordPrefix(1), + NewKeyPrefix: tablecodec.GenTableRecordPrefix(100), + }, + }, + }, + 2: { + Data: []*import_sstpb.RewriteRule{ + { + OldKeyPrefix: tablecodec.GenTableRecordPrefix(2), + NewKeyPrefix: tablecodec.GenTableRecordPrefix(200), + }, + }, + }, + } + oriRegions := [][]byte{ + {}, + codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(100)), + codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(200)), + codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(402)), + } + mockPDCli := split.NewMockPDClientForSplit() + mockPDCli.SetRegions(oriRegions) + client := split.NewClient(mockPDCli, nil, nil, 100, 4) + helper := logsplit.NewLogSplitHelper(rules, client, 4*units.MiB, 400) + + helper.Merge(fakeFile(1, 100, 100, 100)) + helper.Merge(fakeFile(1, 200, 2*units.MiB, 200)) + helper.Merge(fakeFile(2, 100, 3*units.MiB, 300)) + helper.Merge(fakeFile(3, 100, 10*units.MiB, 100000)) + // different regions, no split happens + err := helper.Split(ctx) + require.NoError(t, err) + regions, err := mockPDCli.ScanRegions(ctx, []byte{}, []byte{}, 0) + require.NoError(t, err) + require.Len(t, regions, 3) + require.Equal(t, []byte{}, regions[0].Meta.StartKey) + require.Equal(t, codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(100)), regions[1].Meta.StartKey) + require.Equal(t, codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(200)), regions[2].Meta.StartKey) + require.Equal(t, codec.EncodeBytes(nil, tablecodec.EncodeTablePrefix(402)), regions[2].Meta.EndKey) + + helper.Merge(fakeFile(1, 300, 3*units.MiB, 10)) + helper.Merge(fakeFile(1, 400, 4*units.MiB, 10)) + // trigger to split regions for table 1 + err = helper.Split(ctx) + require.NoError(t, err) + regions, err = mockPDCli.ScanRegions(ctx, []byte{}, []byte{}, 0) + require.NoError(t, err) + require.Len(t, regions, 4) + require.Equal(t, fakeRowKey(100, 400), kv.Key(regions[1].Meta.EndKey)) +} diff --git a/br/pkg/restore/internal/log_split/sum_sorted_test.go b/br/pkg/restore/internal/log_split/sum_sorted_test.go index 1810d31029ed7..634ed93f003b1 100644 --- a/br/pkg/restore/internal/log_split/sum_sorted_test.go +++ b/br/pkg/restore/internal/log_split/sum_sorted_test.go @@ -2,6 +2,7 @@ package logsplit_test import ( + "fmt" "testing" logsplit "github.com/pingcap/tidb/br/pkg/restore/internal/log_split" @@ -25,10 +26,15 @@ func mb(b uint64) logsplit.Value { } } +func exportString(startKey, endKey, size string, number int) string { + return fmt.Sprintf("([%s, %s), %s MB, %d)", startKey, endKey, size, number) +} + func TestSumSorted(t *testing.T) { cases := []struct { values []logsplit.Valued result []uint64 + strs []string }{ { values: []logsplit.Valued{ @@ -37,6 +43,11 @@ func TestSumSorted(t *testing.T) { v("d", "g", mb(100)), }, result: []uint64{0, 250, 25, 75, 50, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("64", "67", "100.00", 100), + }, }, { values: []logsplit.Valued{ @@ -45,6 +56,11 @@ func TestSumSorted(t *testing.T) { v("d", "f", mb(100)), }, result: []uint64{0, 250, 25, 125, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("64", "66", "100.00", 100), + }, }, { values: []logsplit.Valued{ @@ -53,6 +69,11 @@ func TestSumSorted(t *testing.T) { v("c", "f", mb(100)), }, result: []uint64{0, 250, 150, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + }, }, { values: []logsplit.Valued{ @@ -62,6 +83,12 @@ func TestSumSorted(t *testing.T) { v("da", "db", mb(100)), }, result: []uint64{0, 250, 50, 150, 50, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + }, }, { values: []logsplit.Valued{ @@ -72,6 +99,13 @@ func TestSumSorted(t *testing.T) { v("cb", "db", mb(100)), }, result: []uint64{0, 250, 25, 75, 200, 50, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("6362", "6462", "100.00", 100), + }, }, { values: []logsplit.Valued{ @@ -82,6 +116,13 @@ func TestSumSorted(t *testing.T) { v("cb", "f", mb(150)), }, result: []uint64{0, 250, 25, 75, 200, 100, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("6362", "66", "150.00", 150), + }, }, { values: []logsplit.Valued{ @@ -92,6 +133,13 @@ func TestSumSorted(t *testing.T) { v("cb", "df", mb(150)), }, result: []uint64{0, 250, 25, 75, 200, 75, 25, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("6362", "6466", "150.00", 150), + }, }, { values: []logsplit.Valued{ @@ -102,6 +150,13 @@ func TestSumSorted(t *testing.T) { v("cb", "df", mb(150)), }, result: []uint64{0, 250, 25, 75, 200, 75, 25, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("6362", "6466", "150.00", 150), + }, }, { values: []logsplit.Valued{ @@ -112,6 +167,13 @@ func TestSumSorted(t *testing.T) { v("c", "df", mb(150)), }, result: []uint64{0, 250, 100, 200, 75, 25, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("63", "6466", "150.00", 150), + }, }, { values: []logsplit.Valued{ @@ -122,12 +184,20 @@ func TestSumSorted(t *testing.T) { v("c", "f", mb(150)), }, result: []uint64{0, 250, 100, 200, 100, 0}, + strs: []string{ + exportString("61", "66", "100.00", 100), + exportString("61", "63", "200.00", 200), + exportString("63", "66", "100.00", 100), + exportString("6461", "6462", "100.00", 100), + exportString("63", "66", "150.00", 150), + }, }, } for _, ca := range cases { full := logsplit.NewSplitHelper() - for _, v := range ca.values { + for i, v := range ca.values { + require.Equal(t, ca.strs[i], v.String()) full.Merge(v) } diff --git a/br/pkg/restore/internal/prealloc_db/BUILD.bazel b/br/pkg/restore/internal/prealloc_db/BUILD.bazel index c5205aea23179..f6721bde62373 100644 --- a/br/pkg/restore/internal/prealloc_db/BUILD.bazel +++ b/br/pkg/restore/internal/prealloc_db/BUILD.bazel @@ -27,13 +27,17 @@ go_test( timeout = "short", srcs = ["db_test.go"], flaky = True, - shard_count = 4, + shard_count = 8, deps = [ ":prealloc_db", "//br/pkg/gluetidb", "//br/pkg/metautil", "//br/pkg/restore", + "//br/pkg/restore/internal/prealloc_table_id", "//br/pkg/utiltest", + "//pkg/domain", + "//pkg/kv", + "//pkg/meta", "//pkg/meta/autoid", "//pkg/parser/model", "//pkg/parser/mysql", diff --git a/br/pkg/restore/internal/prealloc_db/db_test.go b/br/pkg/restore/internal/prealloc_db/db_test.go index 7b2f11dd21ef7..7cb37c2fa50cc 100644 --- a/br/pkg/restore/internal/prealloc_db/db_test.go +++ b/br/pkg/restore/internal/prealloc_db/db_test.go @@ -4,15 +4,21 @@ package preallocdb_test import ( "context" + "fmt" "math" "strconv" + "sync" "testing" "github.com/pingcap/tidb/br/pkg/gluetidb" "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/restore" preallocdb "github.com/pingcap/tidb/br/pkg/restore/internal/prealloc_db" + prealloctableid "github.com/pingcap/tidb/br/pkg/restore/internal/prealloc_table_id" "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -99,6 +105,270 @@ func TestRestoreAutoIncID(t *testing.T) { require.Equal(t, uint64(globalAutoID+300), autoIncID) } +var createTableSQLs []string = []string{ + "create table `test`.`t1` (id int);", + "create table `test`.`t2` (id int, created_at TIMESTAMP) TTL = `created_at` + INTERVAL 3 MONTH;", + "create sequence `test`.`t3` start 3 increment 2 minvalue 2 maxvalue 10 cache 3;", + "create sequence `test`.`t4` start 3 increment 2 minvalue 2 maxvalue 10 cache 3 cycle;", + "create view `test`.`t5` as select * from `test`.`t1`;", + "create table `test`.`t6` (id int, store_id INT NOT NULL) PARTITION BY RANGE (store_id) (" + + "PARTITION p0 VALUES LESS THAN (6)," + + "PARTITION p1 VALUES LESS THAN (11)," + + "PARTITION p2 VALUES LESS THAN MAXVALUE);", + "create sequence `test`.`t7` start 3 increment -2 minvalue 2 maxvalue 10 cache 3 cycle;", +} + +var insertTableSQLs []string = []string{ + //"SELECT NEXTVAL(`test`.`t3`);", +} + +var checkTableSQLs = []func(t *testing.T, tk *testkit.TestKit, prefix string) error{ + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t1 + sql := fmt.Sprintf("show create table `test`.`%s1`;", prefix) + tk.MustQuery(sql) + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t2 + sql := fmt.Sprintf("show create table `test`.`%s2`;", prefix) + rows := tk.MustQuery(sql).Rows() + require.Contains(t, rows[0][1], "TTL_ENABLE='OFF'") + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t3 + sql := fmt.Sprintf("SELECT NEXTVAL(`test`.`%s3`);", prefix) + rows := tk.MustQuery(sql).Rows() + require.Contains(t, rows[0][0], "3") + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t4 + sql := fmt.Sprintf("SELECT NEXTVAL(`test`.`%s4`);", prefix) + rows := tk.MustQuery(sql).Rows() + require.Contains(t, rows[0][0], "4") + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t5 + sql := fmt.Sprintf("show create table `test`.`%s5`;", prefix) + tk.MustQuery(sql) + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t6 + sql := fmt.Sprintf("show create table `test`.`%s6`;", prefix) + tk.MustQuery(sql) + return nil + }, + func(t *testing.T, tk *testkit.TestKit, prefix string) error { // test.t7 + sql := fmt.Sprintf("SELECT NEXTVAL(`test`.`%s7`);", prefix) + rows := tk.MustQuery(sql).Rows() + require.Contains(t, rows[0][0], "0") + return nil + }, +} + +func prepareAllocTables( + ctx context.Context, + t *testing.T, + db *preallocdb.DB, + dom *domain.Domain, +) (tableInfos []*metautil.Table) { + for _, sql := range createTableSQLs { + err := db.Session().Execute(ctx, sql) + require.NoError(t, err) + } + for _, sql := range insertTableSQLs { + err := db.Session().Execute(ctx, sql) + require.NoError(t, err) + } + + info, err := dom.GetSnapshotInfoSchema(math.MaxUint64) + require.NoError(t, err) + dbInfo, exists := info.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + tableInfos = make([]*metautil.Table, 0, 4) + for i := 1; i <= len(createTableSQLs); i += 1 { + tableName := fmt.Sprintf("t%d", i) + tableInfo, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr(tableName)) + require.NoError(t, err) + tableInfos = append(tableInfos, &metautil.Table{ + DB: dbInfo.Clone(), + Info: tableInfo.Meta().Clone(), + }) + } + + return tableInfos +} + +func cloneTableInfos( + ctx context.Context, + t *testing.T, + db *preallocdb.DB, + dom *domain.Domain, + prefix string, + originTableInfos []*metautil.Table, +) (tableInfos []*metautil.Table) { + // register preallocated ids + var ids *prealloctableid.PreallocIDs + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBR) + tableInfos = make([]*metautil.Table, 0, len(originTableInfos)) + err := kv.RunInNewTxn(ctx, dom.Store(), true, func(_ context.Context, txn kv.Transaction) error { + allocater := meta.NewMeta(txn) + id, e := allocater.GetGlobalID() + if e != nil { + return e + } + + // resign the table id + for i := int64(0); i < int64(len(createTableSQLs)); i += 1 { + newTableInfo := originTableInfos[i].Info.Clone() + newTableInfo.ID = id + i + 1 + newTableInfo.Name = model.NewCIStr(fmt.Sprintf("%s%d", prefix, i+1)) + tableInfos = append(tableInfos, &metautil.Table{ + DB: originTableInfos[i].DB.Clone(), + Info: newTableInfo, + }) + } + + ids = prealloctableid.New(tableInfos) + return ids.Alloc(allocater) + }) + require.NoError(t, err) + db.RegisterPreallocatedIDs(ids) + return tableInfos +} + +func fakePolicyInfo(ident byte) *model.PolicyInfo { + id := int64(ident) + uid := uint64(ident) + str := string(ident) + cistr := model.NewCIStr(str) + return &model.PolicyInfo{ + PlacementSettings: &model.PlacementSettings{ + Followers: uid, + }, + ID: id, + Name: cistr, + State: model.StatePublic, + } +} + +func TestPolicyMode(t *testing.T) { + ctx := context.Background() + s := utiltest.CreateRestoreSchemaSuite(t) + tk := testkit.NewTestKit(t, s.Mock.Storage) + tk.MustExec("use test") + tk.MustExec("set @@sql_mode=''") + tk.MustExec("drop table if exists `t`;") + // Test SQL Mode + db, supportPolicy, err := preallocdb.NewDB(gluetidb.New(), s.Mock.Storage, "STRICT") + require.NoError(t, err) + require.True(t, supportPolicy) + defer db.Close() + + // Prepare the tables + oriTableInfos := prepareAllocTables(ctx, t, db, s.Mock.Domain) + tableInfos := cloneTableInfos(ctx, t, db, s.Mock.Domain, "tt", oriTableInfos) + + // Prepare policy map + policyMap := &sync.Map{} + fakepolicy1 := fakePolicyInfo(1) + fakepolicy2 := fakePolicyInfo(2) + policyMap.Store(fakepolicy1.Name.L, fakepolicy1) + policyMap.Store(fakepolicy2.Name.L, fakepolicy2) + + tableInfos[0].Info.PlacementPolicyRef = &model.PolicyRefInfo{ + ID: fakepolicy1.ID, + Name: fakepolicy1.Name, + } + tableInfos[5].Info.Partition.Definitions[0].PlacementPolicyRef = &model.PolicyRefInfo{ + ID: fakepolicy2.ID, + Name: fakepolicy2.Name, + } + err = db.CreateTables(ctx, tableInfos, nil, true, policyMap) + require.NoError(t, err) + for _, checkFn := range checkTableSQLs { + checkFn(t, tk, "tt") + } + + // clone again to test db.CreateTable + tableInfos = cloneTableInfos(ctx, t, db, s.Mock.Domain, "ttt", oriTableInfos) + + // Prepare policy map + policyMap = &sync.Map{} + fakepolicy1 = fakePolicyInfo(1) + fakepolicy2 = fakePolicyInfo(2) + policyMap.Store(fakepolicy1.Name.L, fakepolicy1) + policyMap.Store(fakepolicy2.Name.L, fakepolicy2) + tableInfos[0].Info.PlacementPolicyRef = &model.PolicyRefInfo{ + ID: fakepolicy1.ID, + Name: fakepolicy1.Name, + } + tableInfos[5].Info.Partition.Definitions[0].PlacementPolicyRef = &model.PolicyRefInfo{ + ID: fakepolicy2.ID, + Name: fakepolicy2.Name, + } + for i := 0; i < len(createTableSQLs); i += 1 { + err = db.CreateTable(ctx, tableInfos[i], nil, true, policyMap) + require.NoError(t, err) + } + + // test db.CreateDatabase + // Prepare policy map + policyMap = &sync.Map{} + fakepolicy1 = fakePolicyInfo(1) + policyMap.Store(fakepolicy1.Name.L, fakepolicy1) + err = db.CreateDatabase(ctx, &model.DBInfo{ + ID: 20000, + Name: model.NewCIStr("test_db"), + Charset: "utf8mb4", + Collate: "utf8mb4_bin", + State: model.StatePublic, + PlacementPolicyRef: &model.PolicyRefInfo{ + ID: fakepolicy1.ID, + Name: fakepolicy1.Name, + }, + }, true, policyMap) + require.NoError(t, err) +} + +func TestUpdateMetaVersion(t *testing.T) { + ctx := context.Background() + s := utiltest.CreateRestoreSchemaSuite(t) + tk := testkit.NewTestKit(t, s.Mock.Storage) + tk.MustExec("use test") + tk.MustExec("set @@sql_mode=''") + tk.MustExec("drop table if exists `t`;") + + // Test SQL Mode + db, supportPolicy, err := preallocdb.NewDB(gluetidb.New(), s.Mock.Storage, "STRICT") + require.NoError(t, err) + require.True(t, supportPolicy) + defer db.Close() + + db.Session().Execute(ctx, "create table test.t (id int);") + db.Session().Execute(ctx, "analyze table test.t;") + db.Session().Execute(ctx, "insert into test.t values (1),(2),(3);") + info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) + require.NoError(t, err) + tableInfo, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + restoreTS := uint64(0) + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBR) + err = kv.RunInNewTxn(ctx, s.Mock.Domain.Store(), true, func(_ context.Context, txn kv.Transaction) error { + restoreTS = txn.StartTS() + return nil + }) + require.NoError(t, err) + tableID := tableInfo.Meta().ID + err = db.UpdateStatsMeta(ctx, tableID, restoreTS, 3) + require.NoError(t, err) + + rows := tk.MustQuery("select version, table_id, modify_count, count, snapshot from mysql.stats_meta;").Rows() + require.Equal(t, fmt.Sprintf("%d", restoreTS), rows[0][0]) + require.Equal(t, fmt.Sprintf("%d", tableID), rows[0][1]) + require.Equal(t, "0", rows[0][2]) + require.Equal(t, "3", rows[0][3]) + require.Equal(t, fmt.Sprintf("%d", restoreTS), rows[0][4]) +} + func TestCreateTablesInDb(t *testing.T) { s := utiltest.CreateRestoreSchemaSuite(t) info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) @@ -136,6 +406,59 @@ func TestCreateTablesInDb(t *testing.T) { require.NoError(t, err) } +func TestDDLJobMap(t *testing.T) { + ctx := context.Background() + s := utiltest.CreateRestoreSchemaSuite(t) + tk := testkit.NewTestKit(t, s.Mock.Storage) + tk.MustExec("use test") + tk.MustExec("set @@sql_mode=''") + + db, supportPolicy, err := preallocdb.NewDB(gluetidb.New(), s.Mock.Storage, "STRICT") + require.NoError(t, err) + require.True(t, supportPolicy) + defer db.Close() + + db.Session().Execute(ctx, "CREATE TABLE test.t1 (a BIGINT PRIMARY KEY AUTO_RANDOM, b VARCHAR(255));") + db.Session().Execute(ctx, "CREATE TABLE test.t2 (a BIGINT AUTO_RANDOM, b VARCHAR(255), PRIMARY KEY (`a`, `b`));") + db.Session().Execute(ctx, "CREATE TABLE test.t3 (a BIGINT PRIMARY KEY AUTO_INCREMENT, b VARCHAR(255));") + db.Session().Execute(ctx, "CREATE TABLE test.t4 (a BIGINT, b VARCHAR(255));") + db.Session().Execute(ctx, "CREATE TABLE test.t5 (a BIGINT PRIMARY KEY, b VARCHAR(255));") + + info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) + require.NoError(t, err) + dbInfo, exists := info.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + tableInfo1, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + tableInfo2, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + tableInfo3, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) + require.NoError(t, err) + tableInfo4, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) + require.NoError(t, err) + tableInfo5, err := info.TableByName(model.NewCIStr("test"), model.NewCIStr("t5")) + require.NoError(t, err) + + toBeCorrectedTables := map[restore.UniqueTableName]bool{ + {DB: "test", Table: "t1"}: true, + {DB: "test", Table: "t2"}: true, + {DB: "test", Table: "t3"}: true, + {DB: "test", Table: "t4"}: true, + {DB: "test", Table: "t5"}: true, + } + + err = db.CreateTablePostRestore(ctx, &metautil.Table{DB: dbInfo.Clone(), Info: tableInfo1.Meta().Clone()}, toBeCorrectedTables) + require.NoError(t, err) + err = db.CreateTablePostRestore(ctx, &metautil.Table{DB: dbInfo.Clone(), Info: tableInfo2.Meta().Clone()}, toBeCorrectedTables) + require.NoError(t, err) + err = db.CreateTablePostRestore(ctx, &metautil.Table{DB: dbInfo.Clone(), Info: tableInfo3.Meta().Clone()}, toBeCorrectedTables) + require.NoError(t, err) + err = db.CreateTablePostRestore(ctx, &metautil.Table{DB: dbInfo.Clone(), Info: tableInfo4.Meta().Clone()}, toBeCorrectedTables) + require.NoError(t, err) + err = db.CreateTablePostRestore(ctx, &metautil.Table{DB: dbInfo.Clone(), Info: tableInfo5.Meta().Clone()}, toBeCorrectedTables) + require.NoError(t, err) +} + func TestDB_ExecDDL(t *testing.T) { s := utiltest.CreateRestoreSchemaSuite(t) @@ -162,6 +485,69 @@ func TestDB_ExecDDL(t *testing.T) { } } +func TestDB_ExecDDL2(t *testing.T) { + s := utiltest.CreateRestoreSchemaSuite(t) + + ctx := context.Background() + fieldType := types.NewFieldType(8) + fieldType.SetFlen(20) + fieldType.SetCharset("binary") + fieldType.SetCollate("binary") + ddlJobs := []*model.Job{ + { + Type: model.ActionCreateSchema, + Query: "CREATE DATABASE IF NOT EXISTS test_db;", + BinlogInfo: &model.HistoryInfo{ + DBInfo: &model.DBInfo{ + ID: 20000, + Name: model.NewCIStr("test_db"), + Charset: "utf8mb4", + Collate: "utf8mb4_bin", + State: model.StatePublic, + }, + }, + }, + { + SchemaName: "test_db", + Type: model.ActionCreateTable, + Query: "CREATE TABLE test_db.t1 (id BIGINT);", + BinlogInfo: &model.HistoryInfo{ + TableInfo: &model.TableInfo{ + ID: 20000, + Name: model.NewCIStr("t1"), + Charset: "utf8mb4", + Collate: "utf8mb4_bin", + Columns: []*model.ColumnInfo{ + { + ID: 1, + Name: model.NewCIStr("id"), + FieldType: *fieldType, + State: model.StatePublic, + Version: 2, + }, + }, + }, + }, + }, + { + SchemaName: "test_db", + Type: model.ActionAddIndex, + Query: "ALTER TABLE test_db.t1 ADD INDEX i1(id);", + BinlogInfo: &model.HistoryInfo{ + TableInfo: &model.TableInfo{}, + }, + }, + } + + db, _, err := preallocdb.NewDB(gluetidb.New(), s.Mock.Storage, "STRICT") + require.NoError(t, err) + + for _, ddlJob := range ddlJobs { + err = db.ExecDDL(ctx, ddlJob) + assert.NoError(t, err) + } +} + func TestCreateTableConsistent(t *testing.T) { ctx := context.Background() s := utiltest.CreateRestoreSchemaSuite(t) diff --git a/br/pkg/restore/internal/prealloc_table_id/alloc_test.go b/br/pkg/restore/internal/prealloc_table_id/alloc_test.go index ae526c8fd6676..b1e7de8978cf6 100644 --- a/br/pkg/restore/internal/prealloc_table_id/alloc_test.go +++ b/br/pkg/restore/internal/prealloc_table_id/alloc_test.go @@ -31,6 +31,7 @@ func TestAllocator(t *testing.T) { hasAllocatedTo int64 successfullyAllocated []int64 shouldAllocatedTo int64 + msg string } cases := []Case{ @@ -39,24 +40,28 @@ func TestAllocator(t *testing.T) { hasAllocatedTo: 6, successfullyAllocated: []int64{6, 7}, shouldAllocatedTo: 8, + msg: "ID:[6,8)", }, { tableIDs: []int64{4, 6, 9, 2}, hasAllocatedTo: 1, successfullyAllocated: []int64{2, 4, 6, 9}, shouldAllocatedTo: 10, + msg: "ID:[1,10)", }, { tableIDs: []int64{1, 2, 3, 4}, hasAllocatedTo: 5, successfullyAllocated: []int64{}, shouldAllocatedTo: 5, + msg: "ID:empty(end=5)", }, { tableIDs: []int64{1, 2, 5, 6, 1 << 50, 1<<50 + 2479}, hasAllocatedTo: 3, successfullyAllocated: []int64{5, 6}, shouldAllocatedTo: 7, + msg: "ID:[3,7)", }, { tableIDs: []int64{1, 2, 5, 6, 7}, @@ -66,6 +71,7 @@ func TestAllocator(t *testing.T) { partitions: map[int64][]int64{ 7: {8, 9, 10, 11, 12}, }, + msg: "ID:[6,13)", }, { tableIDs: []int64{1, 2, 5, 6, 7, 13}, @@ -75,6 +81,7 @@ func TestAllocator(t *testing.T) { partitions: map[int64][]int64{ 7: {8, 9, 10, 11, 12}, }, + msg: "ID:[9,14)", }, } @@ -98,6 +105,7 @@ func TestAllocator(t *testing.T) { ids := prealloctableid.New(tables) allocator := testAllocator(c.hasAllocatedTo) require.NoError(t, ids.Alloc(&allocator)) + require.Equal(t, c.msg, ids.String()) allocated := make([]int64, 0, len(c.successfullyAllocated)) for _, t := range tables { diff --git a/br/pkg/restore/internal/utils/BUILD.bazel b/br/pkg/restore/internal/utils/BUILD.bazel index 607722774f9d0..9d791b3d5fefd 100644 --- a/br/pkg/restore/internal/utils/BUILD.bazel +++ b/br/pkg/restore/internal/utils/BUILD.bazel @@ -22,7 +22,7 @@ go_test( timeout = "short", srcs = ["split_test.go"], flaky = True, - shard_count = 4, + shard_count = 5, deps = [ ":utils", "//br/pkg/restore/split", diff --git a/br/pkg/restore/internal/utils/split_test.go b/br/pkg/restore/internal/utils/split_test.go index a1a45e5513313..4eeacb69f61c9 100644 --- a/br/pkg/restore/internal/utils/split_test.go +++ b/br/pkg/restore/internal/utils/split_test.go @@ -31,6 +31,15 @@ func TestScanEmptyRegion(t *testing.T) { require.NoError(t, err) } +func TestSplitEmptyRegion(t *testing.T) { + mockPDCli := split.NewMockPDClientForSplit() + mockPDCli.SetRegions([][]byte{{}, {12}, {34}, {}}) + client := split.NewClient(mockPDCli, nil, nil, 100, 4) + regionSplitter := utils.NewRegionSplitter(client) + err := regionSplitter.ExecuteSplit(context.Background(), nil) + require.NoError(t, err) +} + // region: [, aay), [aay, bba), [bba, bbh), [bbh, cca), [cca, ) // range: [aaa, aae), [aae, aaz), [ccd, ccf), [ccf, ccj) // rewrite rules: aa -> xx, cc -> bb @@ -206,6 +215,14 @@ func TestSortRange(t *testing.T) { {StartKey: []byte("xxa"), EndKey: []byte("xxe"), Files: nil}, {StartKey: []byte("xxe"), EndKey: []byte("xxz"), Files: nil}, }) + + // overlap ranges + ranges4 := []rtree.Range{ + {StartKey: []byte("aaa"), EndKey: []byte("aae")}, + {StartKey: []byte("aaa"), EndKey: []byte("aaz")}, + } + _, err = utils.SortRanges(ranges4) + require.Error(t, err) } func rangeEquals(t *testing.T, obtained, expected []rtree.Range) { diff --git a/br/pkg/restore/log_client/BUILD.bazel b/br/pkg/restore/log_client/BUILD.bazel index 25431dd3e6d4a..f6c932a04a6ea 100644 --- a/br/pkg/restore/log_client/BUILD.bazel +++ b/br/pkg/restore/log_client/BUILD.bazel @@ -82,11 +82,12 @@ go_test( ], embed = [":log_client"], flaky = True, - shard_count = 36, + shard_count = 38, deps = [ "//br/pkg/errors", "//br/pkg/gluetidb", "//br/pkg/mock", + "//br/pkg/restore/internal/import_client", "//br/pkg/restore/split", "//br/pkg/restore/utils", "//br/pkg/storage", @@ -109,6 +110,7 @@ go_test( "@com_github_pingcap_kvproto//pkg/pdpb", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", + "@com_github_tikv_pd_client//:client", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//keepalive", "@org_golang_google_grpc//status", diff --git a/br/pkg/restore/log_client/export_test.go b/br/pkg/restore/log_client/export_test.go index c28d46c066798..d0e4f1e2dd528 100644 --- a/br/pkg/restore/log_client/export_test.go +++ b/br/pkg/restore/log_client/export_test.go @@ -18,6 +18,8 @@ import ( "context" "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils/iter" ) @@ -35,3 +37,29 @@ func (rc *LogFileManager) ReadStreamMeta(ctx context.Context) ([]Meta, error) { } return r.Item, nil } + +func TEST_NewLogFileManager(startTS, restoreTS, shiftStartTS uint64, helper streamMetadataHelper) *LogFileManager { + return &LogFileManager{ + startTS: startTS, + restoreTS: restoreTS, + shiftStartTS: shiftStartTS, + helper: helper, + } +} + +type FakeStreamMetadataHelper struct { + streamMetadataHelper + + Data []byte +} + +func (helper *FakeStreamMetadataHelper) ReadFile( + ctx context.Context, + path string, + offset uint64, + length uint64, + compressionType backuppb.CompressionType, + storage storage.ExternalStorage, +) ([]byte, error) { + return helper.Data[offset : offset+length], nil +} diff --git a/br/pkg/restore/log_client/import_retry_test.go b/br/pkg/restore/log_client/import_retry_test.go index fd4fc32c18317..4234f6fd63d8b 100644 --- a/br/pkg/restore/log_client/import_retry_test.go +++ b/br/pkg/restore/log_client/import_retry_test.go @@ -22,9 +22,11 @@ import ( logclient "github.com/pingcap/tidb/br/pkg/restore/log_client" "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/store/pdtypes" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) @@ -53,6 +55,7 @@ func assertRegions(t *testing.T, regions []*split.RegionInfo, keys ...string) { type TestClient struct { split.SplitClient + pd.Client mu sync.RWMutex stores map[uint64]*metapb.Store @@ -89,6 +92,14 @@ func (c *TestClient) GetAllRegions() map[uint64]*split.RegionInfo { return c.regions } +func (c *TestClient) GetPDClient() *utiltest.FakePDClient { + stores := make([]*metapb.Store, 0, len(c.stores)) + for _, store := range c.stores { + stores = append(stores, store) + } + return utiltest.NewFakePDClient(stores, false, nil) +} + func (c *TestClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { c.mu.RLock() defer c.mu.RUnlock() diff --git a/br/pkg/restore/log_client/import_test.go b/br/pkg/restore/log_client/import_test.go index 04445730cdfa0..af5add181d193 100644 --- a/br/pkg/restore/log_client/import_test.go +++ b/br/pkg/restore/log_client/import_test.go @@ -19,11 +19,16 @@ import ( "testing" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" berrors "github.com/pingcap/tidb/br/pkg/errors" + importclient "github.com/pingcap/tidb/br/pkg/restore/internal/import_client" logclient "github.com/pingcap/tidb/br/pkg/restore/log_client" "github.com/pingcap/tidb/br/pkg/restore/split" + restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" ) @@ -197,3 +202,75 @@ func TestFilterFilesByRegion(t *testing.T) { require.Equal(t, subfile, c.subfiles) } } + +type fakeImportClient struct { + importclient.ImporterClient +} + +func (client *fakeImportClient) ClearFiles( + ctx context.Context, + storeID uint64, + req *import_sstpb.ClearRequest, +) (*import_sstpb.ClearResponse, error) { + return &import_sstpb.ClearResponse{Error: &import_sstpb.Error{Message: req.Prefix}}, nil +} + +func (client *fakeImportClient) CloseGrpcClient() error { return nil } + +func (client *fakeImportClient) ApplyKVFile( + ctx context.Context, + storeID uint64, + req *import_sstpb.ApplyRequest, +) (*import_sstpb.ApplyResponse, error) { + if len(req.Metas) == 0 { + return &import_sstpb.ApplyResponse{}, berrors.ErrKVRangeIsEmpty + } + return &import_sstpb.ApplyResponse{}, nil +} + +func prepareData() (*restoreutils.RewriteRules, []*logclient.LogDataFileInfo) { + rewriteRules := &restoreutils.RewriteRules{ + Data: []*import_sstpb.RewriteRule{ + { + NewKeyPrefix: tablecodec.GenTablePrefix(2), + OldKeyPrefix: tablecodec.GenTablePrefix(1), + }, + { + NewKeyPrefix: tablecodec.GenTablePrefix(511), + OldKeyPrefix: tablecodec.GenTablePrefix(767), + }, + }, + } + + encodeKeyFiles := []*logclient.LogDataFileInfo{ + { + DataFileInfo: &backuppb.DataFileInfo{ + Path: "bakcup.log", + StartKey: codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(1)), + EndKey: codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(1).PrefixNext()), + }, + }, + } + + return rewriteRules, encodeKeyFiles +} + +func TestFileImporter(t *testing.T) { + ctx := context.Background() + metaClient := initTestClient(false) + mockImportClient := &fakeImportClient{} + importer := logclient.NewLogFileImporter(metaClient, mockImportClient, nil) + defer func() { + require.NoError(t, importer.Close()) + }() + + err := importer.ClearFiles(ctx, metaClient.GetPDClient(), "test") + require.NoError(t, err) + + rewriteRules, encodeKeyFiles := prepareData() + err = importer.ImportKVFiles(ctx, encodeKeyFiles, rewriteRules, 1, 1, 1, true) + require.NoError(t, err) + + err = importer.ImportKVFiles(ctx, encodeKeyFiles, rewriteRules, 1, 1, 1, false) + require.NoError(t, err) +} diff --git a/br/pkg/restore/log_client/log_file_manager.go b/br/pkg/restore/log_client/log_file_manager.go index b5a30f44d806c..d29b4f1f7ab8e 100644 --- a/br/pkg/restore/log_client/log_file_manager.go +++ b/br/pkg/restore/log_client/log_file_manager.go @@ -45,6 +45,19 @@ type Meta = *backuppb.Metadata // Log is the metadata of one file recording KV sequences. type Log = *backuppb.DataFileInfo +type streamMetadataHelper interface { + InitCacheEntry(path string, ref int) + ReadFile( + ctx context.Context, + path string, + offset uint64, + length uint64, + compressionType backuppb.CompressionType, + storage storage.ExternalStorage, + ) ([]byte, error) + ParseToMetadata(rawMetaData []byte) (*backuppb.Metadata, error) +} + // LogFileManager is the manager for log files of a certain restoration, // which supports read / filter from the log backup archive with static start TS / restore TS. type LogFileManager struct { @@ -60,7 +73,7 @@ type LogFileManager struct { shiftStartTS uint64 storage storage.ExternalStorage - helper *stream.MetadataHelper + helper streamMetadataHelper metadataDownloadBatchSize uint } diff --git a/br/pkg/restore/log_client/log_file_manager_test.go b/br/pkg/restore/log_client/log_file_manager_test.go index f4c2906a9f03f..82fcf628d0139 100644 --- a/br/pkg/restore/log_client/log_file_manager_test.go +++ b/br/pkg/restore/log_client/log_file_manager_test.go @@ -7,6 +7,7 @@ package logclient_test import ( "context" + "crypto/sha256" "fmt" "math" "os" @@ -23,6 +24,8 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/utils/iter" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -551,3 +554,93 @@ func TestFilterDataFiles(t *testing.T) { } } } + +func encodekv(prefix string, ts uint64, emptyV bool) []byte { + k := fmt.Sprintf("%s_%d", prefix, ts) + v := "any value" + if emptyV { + v = "" + } + kts := codec.EncodeUintDesc([]byte(k), ts) + return stream.EncodeKVEntry(kts, []byte(v)) +} + +func encodekvEntryWithTS(prefix string, ts uint64) *logclient.KvEntryWithTS { + k := fmt.Sprintf("%s_%d", prefix, ts) + v := "any value" + kts := codec.EncodeUintDesc([]byte(k), ts) + return &logclient.KvEntryWithTS{ + E: kv.Entry{ + Key: kts, + Value: []byte(v), + }, + Ts: ts, + } +} + +func generateKvData() ([]byte, logclient.Log) { + buff := make([]byte, 0) + rangeLength := uint64(0) + buff = append(buff, encodekv("mDDLHistory", 10, false)...) + buff = append(buff, encodekv("mDDLHistory", 10, true)...) + rangeOffset := uint64(len(buff)) + buff = append(buff, encodekv("mDDLHistory", 21, false)...) + buff = append(buff, encodekv("mDDLHistory", 22, true)...) + buff = append(buff, encodekv("mDDL", 27, false)...) + buff = append(buff, encodekv("mDDL", 28, true)...) + buff = append(buff, encodekv("mDDL", 37, false)...) + buff = append(buff, encodekv("mDDL", 38, true)...) + buff = append(buff, encodekv("mDDLHistory", 45, false)...) + buff = append(buff, encodekv("mDDLHistory", 45, true)...) + buff = append(buff, encodekv("mDDL", 50, false)...) + buff = append(buff, encodekv("mDDL", 50, true)...) + buff = append(buff, encodekv("mTable", 52, false)...) + buff = append(buff, encodekv("mTable", 52, true)...) + buff = append(buff, encodekv("mDDL", 65, false)...) + buff = append(buff, encodekv("mDDL", 65, true)...) + buff = append(buff, encodekv("mDDLHistory", 80, false)...) + buff = append(buff, encodekv("mDDLHistory", 80, true)...) + rangeLength = uint64(len(buff)) - rangeOffset + buff = append(buff, encodekv("mDDL", 90, false)...) + buff = append(buff, encodekv("mDDL", 90, true)...) + + sha256 := sha256.Sum256(buff[rangeOffset : rangeOffset+rangeLength]) + return buff, &backuppb.DataFileInfo{ + Sha256: sha256[:], + RangeOffset: rangeOffset, + RangeLength: rangeLength, + } +} + +func TestReadAllEntries(t *testing.T) { + ctx := context.Background() + data, file := generateKvData() + fm := logclient.TEST_NewLogFileManager(35, 75, 25, &logclient.FakeStreamMetadataHelper{Data: data}) + { + file.Cf = stream.WriteCF + kvEntries, nextKvEntries, err := fm.ReadAllEntries(ctx, file, 50) + require.NoError(t, err) + require.Equal(t, []*logclient.KvEntryWithTS{ + encodekvEntryWithTS("mDDL", 37), + encodekvEntryWithTS("mDDLHistory", 45), + }, kvEntries) + require.Equal(t, []*logclient.KvEntryWithTS{ + encodekvEntryWithTS("mDDL", 50), + encodekvEntryWithTS("mDDL", 65), + }, nextKvEntries) + } + { + file.Cf = stream.DefaultCF + kvEntries, nextKvEntries, err := fm.ReadAllEntries(ctx, file, 50) + require.NoError(t, err) + require.Equal(t, []*logclient.KvEntryWithTS{ + encodekvEntryWithTS("mDDL", 27), + encodekvEntryWithTS("mDDL", 37), + encodekvEntryWithTS("mDDLHistory", 45), + }, kvEntries) + require.Equal(t, []*logclient.KvEntryWithTS{ + encodekvEntryWithTS("mDDL", 50), + encodekvEntryWithTS("mDDL", 65), + }, nextKvEntries) + } +} diff --git a/br/pkg/restore/misc_test.go b/br/pkg/restore/misc_test.go index 3e63753b0cb36..7d9fe6699505e 100644 --- a/br/pkg/restore/misc_test.go +++ b/br/pkg/restore/misc_test.go @@ -15,16 +15,37 @@ package restore_test import ( + "context" "math" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/model" "github.com/stretchr/testify/require" ) +func TestTransferBoolToValue(t *testing.T) { + require.Equal(t, "ON", restore.TransferBoolToValue(true)) + require.Equal(t, "OFF", restore.TransferBoolToValue(false)) +} + +func TestGetTableSchema(t *testing.T) { + m, err := mock.NewCluster() + require.Nil(t, err) + defer m.Stop() + dom := m.Domain + + _, err = restore.GetTableSchema(dom, model.NewCIStr("test"), model.NewCIStr("tidb")) + require.Error(t, err) + tableInfo, err := restore.GetTableSchema(dom, model.NewCIStr("mysql"), model.NewCIStr("tidb")) + require.NoError(t, err) + require.Equal(t, model.NewCIStr("tidb"), tableInfo.Name) +} + func TestGetExistedUserDBs(t *testing.T) { m, err := mock.NewCluster() require.Nil(t, err) @@ -73,3 +94,30 @@ func TestGetExistedUserDBs(t *testing.T) { dbs = restore.GetExistedUserDBs(dom) require.Equal(t, 2, len(dbs)) } + +func TestGetTSWithRetry(t *testing.T) { + t.Run("PD leader is healthy:", func(t *testing.T) { + retryTimes := -1000 + pDClient := utiltest.NewFakePDClient(nil, false, &retryTimes) + _, err := restore.GetTSWithRetry(context.Background(), pDClient) + require.NoError(t, err) + }) + + t.Run("PD leader failure:", func(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/utils/set-attempt-to-one", "1*return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/utils/set-attempt-to-one")) + }() + retryTimes := -1000 + pDClient := utiltest.NewFakePDClient(nil, true, &retryTimes) + _, err := restore.GetTSWithRetry(context.Background(), pDClient) + require.Error(t, err) + }) + + t.Run("PD leader switch successfully", func(t *testing.T) { + retryTimes := 0 + pDClient := utiltest.NewFakePDClient(nil, true, &retryTimes) + _, err := restore.GetTSWithRetry(context.Background(), pDClient) + require.NoError(t, err) + }) +} diff --git a/br/pkg/restore/snap_client/BUILD.bazel b/br/pkg/restore/snap_client/BUILD.bazel index d8ff0e9f84bf0..6f01151d99bc6 100644 --- a/br/pkg/restore/snap_client/BUILD.bazel +++ b/br/pkg/restore/snap_client/BUILD.bazel @@ -77,6 +77,7 @@ go_test( srcs = [ "batcher_test.go", "client_test.go", + "context_manager_test.go", "export_test.go", "import_test.go", "main_test.go", @@ -85,7 +86,7 @@ go_test( ], embed = [":snap_client"], flaky = True, - shard_count = 17, + shard_count = 23, deps = [ "//br/pkg/errors", "//br/pkg/glue", @@ -112,9 +113,11 @@ go_test( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/import_sstpb", + "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", + "@com_github_tikv_pd_client//:client", "@org_golang_x_exp//slices", "@org_uber_go_goleak//:goleak", "@org_uber_go_zap//:zap", diff --git a/br/pkg/restore/snap_client/context_manager.go b/br/pkg/restore/snap_client/context_manager.go index 649f5b8b87980..294f774630db6 100644 --- a/br/pkg/restore/snap_client/context_manager.go +++ b/br/pkg/restore/snap_client/context_manager.go @@ -24,6 +24,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/conn" @@ -241,6 +242,10 @@ func (manager *brContextManager) waitPlacementSchedule(ctx context.Context, tabl } log.Info("start waiting placement schedule") ticker := time.NewTicker(time.Second * 10) + failpoint.Inject("wait-placement-schedule-quicker-ticker", func() { + ticker.Stop() + ticker = time.NewTicker(time.Millisecond * 500) + }) defer ticker.Stop() for { select { diff --git a/br/pkg/restore/snap_client/context_manager_test.go b/br/pkg/restore/snap_client/context_manager_test.go new file mode 100644 index 0000000000000..c13326d21653a --- /dev/null +++ b/br/pkg/restore/snap_client/context_manager_test.go @@ -0,0 +1,259 @@ +// Copyright 2024 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 snapclient_test + +import ( + "context" + "testing" + + "github.com/pingcap/failpoint" + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/metautil" + snapclient "github.com/pingcap/tidb/br/pkg/restore/snap_client" + restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" + "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/util/codec" + "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" +) + +func generateTables() []snapclient.CreatedTable { + return []snapclient.CreatedTable{ + { + Table: &model.TableInfo{ + ID: 1, + }, + OldTable: &metautil.Table{ + DB: &model.DBInfo{Name: model.NewCIStr("test")}, + Info: &model.TableInfo{Name: model.NewCIStr("t1")}, + }, + }, + { + Table: &model.TableInfo{ + ID: 100, + }, + OldTable: &metautil.Table{ + DB: &model.DBInfo{Name: model.NewCIStr("test")}, + Info: &model.TableInfo{Name: model.NewCIStr("t100")}, + }, + }, + } +} + +func TestContextManagerOfflineLeave(t *testing.T) { + ctx := context.Background() + brContextManager, err := snapclient.NewBRContextManager(ctx, nil, nil, nil, false) + require.NoError(t, err) + tables := generateTables() + err = brContextManager.Enter(ctx, tables) + require.NoError(t, err) + err = brContextManager.Leave(ctx, tables) + require.NoError(t, err) + brContextManager.Close(ctx) +} + +func TestContextManagerOfflineClose(t *testing.T) { + ctx := context.Background() + brContextManager, err := snapclient.NewBRContextManager(ctx, nil, nil, nil, false) + require.NoError(t, err) + tables := generateTables() + err = brContextManager.Enter(ctx, tables) + require.NoError(t, err) + brContextManager.Close(ctx) +} + +func TestContextManagerOnlineNoStores(t *testing.T) { + ctx := context.Background() + stores := []*metapb.Store{ + { + Id: 1, + State: metapb.StoreState_Up, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tiflash", + }, + }, + }, + { + Id: 2, + State: metapb.StoreState_Offline, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tikv", + }, + }, + }, + } + + pdClient := utiltest.NewFakePDClient(stores, false, nil) + pdHTTPCli := utiltest.NewFakePDHTTPClient() + brContextManager, err := snapclient.NewBRContextManager(ctx, pdClient, pdHTTPCli, nil, true) + require.NoError(t, err) + tables := generateTables() + err = brContextManager.Enter(ctx, tables) + require.NoError(t, err) + err = brContextManager.Leave(ctx, tables) + require.NoError(t, err) + brContextManager.Close(ctx) +} + +func generateRegions() []*pd.Region { + return []*pd.Region{ + { + Meta: &metapb.Region{ + Id: 0, + StartKey: []byte(""), + EndKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(0)), + Peers: []*metapb.Peer{{StoreId: 1}, {StoreId: 2}}, + }, + Leader: &metapb.Peer{StoreId: 1}, + }, + { + Meta: &metapb.Region{ + Id: 1, + StartKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(0)), + EndKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(1)), + Peers: []*metapb.Peer{{StoreId: 1}, {StoreId: 2}}, + }, + Leader: &metapb.Peer{StoreId: 1}, + }, + { + Meta: &metapb.Region{ + Id: 2, + StartKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(1)), + EndKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(2)), + Peers: []*metapb.Peer{{StoreId: 1}, {StoreId: 2}}, + }, + Leader: &metapb.Peer{StoreId: 1}, + }, + { + Meta: &metapb.Region{ + Id: 3, + StartKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(2)), + EndKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(200)), + Peers: []*metapb.Peer{{StoreId: 1}, {StoreId: 2}}, + }, + Leader: &metapb.Peer{StoreId: 1}, + }, + { + Meta: &metapb.Region{ + Id: 4, + StartKey: codec.EncodeBytes([]byte{}, tablecodec.EncodeTablePrefix(200)), + EndKey: []byte(""), + Peers: []*metapb.Peer{{StoreId: 1}, {StoreId: 2}}, + }, + Leader: &metapb.Peer{StoreId: 1}, + }, + } +} + +func generateFiles() ([]*backuppb.File, *restoreutils.RewriteRules) { + files := make([]*backuppb.File, 0, 10) + for i := 0; i < 10; i += 1 { + files = append(files, &backuppb.File{ + StartKey: tablecodec.EncodeTablePrefix(100), + EndKey: tablecodec.EncodeTablePrefix(100), + }) + } + return files, &restoreutils.RewriteRules{ + Data: []*import_sstpb.RewriteRule{ + { + OldKeyPrefix: tablecodec.EncodeTablePrefix(100), + NewKeyPrefix: tablecodec.EncodeTablePrefix(1), + }, + }, + } +} + +func generateStores() []*metapb.Store { + return []*metapb.Store{ + { + Id: 1, + State: metapb.StoreState_Up, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tiflash", + }, + }, + }, + { + Id: 2, + State: metapb.StoreState_Up, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tikv", + }, + { + Key: snapclient.RestoreLabelKey, + Value: snapclient.RestoreLabelValue, + }, + }, + }, + { + Id: 3, + State: metapb.StoreState_Offline, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tikv", + }, + }, + }, + { + Id: 4, + State: metapb.StoreState_Offline, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tikv", + }, + { + Key: snapclient.RestoreLabelKey, + Value: snapclient.RestoreLabelValue, + }, + }, + }, + } +} + +func TestContextManagerOnlineLeave(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/restore/snap_client/wait-placement-schedule-quicker-ticker", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/restore/snap_client/wait-placement-schedule-quicker-ticker")) + }() + ctx := context.Background() + stores := generateStores() + regions := generateRegions() + + pdClient := utiltest.NewFakePDClient(stores, false, nil) + pdClient.SetRegions(regions) + pdHTTPCli := utiltest.NewFakePDHTTPClient() + brContextManager, err := snapclient.NewBRContextManager(ctx, pdClient, pdHTTPCli, nil, true) + require.NoError(t, err) + tables := generateTables() + err = brContextManager.Enter(ctx, tables) + require.NoError(t, err) + err = brContextManager.Leave(ctx, tables) + require.NoError(t, err) + brContextManager.Close(ctx) +} diff --git a/br/pkg/restore/snap_client/export_test.go b/br/pkg/restore/snap_client/export_test.go index 88d32d61d9c57..c519fd0421200 100644 --- a/br/pkg/restore/snap_client/export_test.go +++ b/br/pkg/restore/snap_client/export_test.go @@ -29,9 +29,13 @@ import ( "golang.org/x/exp/slices" ) -var GetSSTMetaFromFile = getSSTMetaFromFile +var ( + RestoreLabelKey = restoreLabelKey + RestoreLabelValue = restoreLabelValue -var GetKeyRangeByMode = getKeyRangeByMode + GetSSTMetaFromFile = getSSTMetaFromFile + GetKeyRangeByMode = getKeyRangeByMode +) // MockClient create a fake Client used to test. func MockClient(dbs map[string]*metautil.Database) *SnapClient { diff --git a/br/pkg/restore/snap_client/import_test.go b/br/pkg/restore/snap_client/import_test.go index 7ed64c1b96ea8..71d5a758f39c7 100644 --- a/br/pkg/restore/snap_client/import_test.go +++ b/br/pkg/restore/snap_client/import_test.go @@ -15,13 +15,17 @@ package snapclient_test import ( + "context" "testing" 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" + importclient "github.com/pingcap/tidb/br/pkg/restore/internal/import_client" snapclient "github.com/pingcap/tidb/br/pkg/restore/snap_client" restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" + "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" ) @@ -104,3 +108,93 @@ func TestGetSSTMetaFromFile(t *testing.T) { require.Equal(t, "t2abc", string(sstMeta.GetRange().GetStart())) require.Equal(t, "t2\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff", string(sstMeta.GetRange().GetEnd())) } + +type fakeImporterClient struct { + importclient.ImporterClient + + speedLimit map[uint64]uint64 +} + +func newFakeImporterClient() *fakeImporterClient { + return &fakeImporterClient{ + speedLimit: make(map[uint64]uint64), + } +} + +func (client *fakeImporterClient) SetDownloadSpeedLimit( + ctx context.Context, + storeID uint64, + req *import_sstpb.SetDownloadSpeedLimitRequest, +) (*import_sstpb.SetDownloadSpeedLimitResponse, error) { + client.speedLimit[storeID] = req.SpeedLimit + return &import_sstpb.SetDownloadSpeedLimitResponse{}, nil +} + +func (client *fakeImporterClient) CheckMultiIngestSupport(ctx context.Context, stores []uint64) error { + return nil +} + +func (client *fakeImporterClient) CloseGrpcClient() error { + return nil +} + +func (client *fakeImporterClient) DownloadSST( + ctx context.Context, + storeID uint64, + req *import_sstpb.DownloadRequest, +) (*import_sstpb.DownloadResponse, error) { + return &import_sstpb.DownloadResponse{Range: *req.Sst.Range}, nil +} + +func (client *fakeImporterClient) MultiIngest( + ctx context.Context, + storeID uint64, + req *import_sstpb.MultiIngestRequest, +) (*import_sstpb.IngestResponse, error) { + return &import_sstpb.IngestResponse{}, nil +} + +func TestSnapImporter(t *testing.T) { + ctx := context.Background() + splitClient := utiltest.NewFakeSplitClient() + for _, region := range generateRegions() { + splitClient.AppendPdRegion(region) + } + importClient := newFakeImporterClient() + importer, err := snapclient.NewSnapFileImporter(ctx, splitClient, importClient, nil, false, false, generateStores(), snapclient.RewriteModeKeyspace, 10) + require.NoError(t, err) + err = importer.SetDownloadSpeedLimit(ctx, 1, 5) + require.NoError(t, err) + require.Equal(t, uint64(5), importClient.speedLimit[1]) + err = importer.SetRawRange(nil, nil) + require.Error(t, err) + files, rules := generateFiles() + for _, file := range files { + importer.WaitUntilUnblock() + err = importer.ImportSSTFiles(ctx, []*backuppb.File{file}, rules, nil, kvrpcpb.APIVersion_V1) + require.NoError(t, err) + } + err = importer.Close() + require.NoError(t, err) +} + +func TestSnapImporterRaw(t *testing.T) { + ctx := context.Background() + splitClient := utiltest.NewFakeSplitClient() + for _, region := range generateRegions() { + splitClient.AppendPdRegion(region) + } + importClient := newFakeImporterClient() + importer, err := snapclient.NewSnapFileImporter(ctx, splitClient, importClient, nil, true, false, generateStores(), snapclient.RewriteModeKeyspace, 10) + require.NoError(t, err) + err = importer.SetRawRange([]byte(""), []byte("")) + require.NoError(t, err) + files, rules := generateFiles() + for _, file := range files { + importer.WaitUntilUnblock() + err = importer.ImportSSTFiles(ctx, []*backuppb.File{file}, rules, nil, kvrpcpb.APIVersion_V1) + require.NoError(t, err) + } + err = importer.Close() + require.NoError(t, err) +} diff --git a/br/pkg/restore/split/BUILD.bazel b/br/pkg/restore/split/BUILD.bazel index 1d1c05350b8a9..2d7002b493ad2 100644 --- a/br/pkg/restore/split/BUILD.bazel +++ b/br/pkg/restore/split/BUILD.bazel @@ -55,7 +55,7 @@ go_test( ], embed = [":split"], flaky = True, - shard_count = 17, + shard_count = 19, deps = [ "//br/pkg/errors", "//br/pkg/utils", diff --git a/br/pkg/restore/split/split.go b/br/pkg/restore/split/split.go index b41a0ec62fa94..ce6faa90b209c 100644 --- a/br/pkg/restore/split/split.go +++ b/br/pkg/restore/split/split.go @@ -148,8 +148,8 @@ func PaginateScanRegion( return lastRegions, err } -// CheckPartRegionConsistency only checks the continuity of regions and the first region consistency. -func CheckPartRegionConsistency(startKey, endKey []byte, regions []*RegionInfo) error { +// checkPartRegionConsistency only checks the continuity of regions and the first region consistency. +func checkPartRegionConsistency(startKey, endKey []byte, regions []*RegionInfo) error { // current pd can't guarantee the consistency of returned regions if len(regions) == 0 { return errors.Annotatef(berrors.ErrPDBatchScanRegion, @@ -199,7 +199,7 @@ func ScanRegionsWithRetry( return err } - if err = CheckPartRegionConsistency(startKey, endKey, regions); err != nil { + if err = checkPartRegionConsistency(startKey, endKey, regions); err != nil { log.Warn("failed to scan region, retrying", logutil.ShortError(err)) return err } diff --git a/br/pkg/restore/split/split_test.go b/br/pkg/restore/split/split_test.go index 9ca523fe214f4..2250f7a96635c 100644 --- a/br/pkg/restore/split/split_test.go +++ b/br/pkg/restore/split/split_test.go @@ -333,6 +333,7 @@ func TestGetSplitKeyPerRegion(t *testing.T) { []byte("g"), []byte("j"), []byte("l"), + []byte("m"), } sortedRegions := []*RegionInfo{ { @@ -689,3 +690,69 @@ func TestRegionConsistency(t *testing.T) { require.Regexp(t, ca.err, err.Error()) } } + +func regionInfo(startKey, endKey string) *RegionInfo { + return &RegionInfo{ + Region: &metapb.Region{ + StartKey: []byte(startKey), + EndKey: []byte(endKey), + }, + } +} + +func TestSplitCheckPartRegionConsistency(t *testing.T) { + var ( + startKey []byte = []byte("a") + endKey []byte = []byte("f") + err error + ) + err = checkPartRegionConsistency(startKey, endKey, nil) + require.Error(t, err) + err = checkPartRegionConsistency(startKey, endKey, []*RegionInfo{ + regionInfo("b", "c"), + }) + require.Error(t, err) + err = checkPartRegionConsistency(startKey, endKey, []*RegionInfo{ + regionInfo("a", "c"), + regionInfo("d", "e"), + }) + require.Error(t, err) + err = checkPartRegionConsistency(startKey, endKey, []*RegionInfo{ + regionInfo("a", "c"), + regionInfo("c", "d"), + }) + require.NoError(t, err) + err = checkPartRegionConsistency(startKey, endKey, []*RegionInfo{ + regionInfo("a", "c"), + regionInfo("c", "d"), + regionInfo("d", "f"), + }) + require.NoError(t, err) + err = checkPartRegionConsistency(startKey, endKey, []*RegionInfo{ + regionInfo("a", "c"), + regionInfo("c", "z"), + }) + require.NoError(t, err) +} + +func TestScanRegionsWithRetry(t *testing.T) { + ctx := context.Background() + mockPDClient := NewMockPDClientForSplit() + mockClient := &pdClient{ + client: mockPDClient, + } + + { + _, err := ScanRegionsWithRetry(ctx, mockClient, []byte("1"), []byte("0"), 0) + require.Error(t, err) + } + + { + mockPDClient.SetRegions([][]byte{{}, []byte("1"), []byte("2"), []byte("3"), []byte("4"), {}}) + regions, err := ScanRegionsWithRetry(ctx, mockClient, []byte("1"), []byte("3"), 0) + require.NoError(t, err) + require.Len(t, regions, 2) + require.Equal(t, []byte("1"), regions[0].Region.StartKey) + require.Equal(t, []byte("2"), regions[1].Region.StartKey) + } +} diff --git a/br/pkg/restore/utils/BUILD.bazel b/br/pkg/restore/utils/BUILD.bazel index ec0b08d320555..8b0376d89c641 100644 --- a/br/pkg/restore/utils/BUILD.bazel +++ b/br/pkg/restore/utils/BUILD.bazel @@ -34,24 +34,21 @@ go_test( "rewrite_rule_test.go", ], flaky = True, - shard_count = 9, + shard_count = 14, deps = [ ":utils", "//br/pkg/conn", "//br/pkg/errors", - "//br/pkg/restore", - "//br/pkg/restore/split", "//br/pkg/rtree", - "//br/pkg/utiltest", + "//pkg/kv", + "//pkg/parser/model", "//pkg/sessionctx/stmtctx", "//pkg/tablecodec", "//pkg/types", "//pkg/util/codec", "@com_github_pingcap_errors//:errors", - "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/import_sstpb", - "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_stretchr_testify//require", ], ) diff --git a/br/pkg/restore/utils/misc_test.go b/br/pkg/restore/utils/misc_test.go index f7314829f3af9..e094774224fcd 100644 --- a/br/pkg/restore/utils/misc_test.go +++ b/br/pkg/restore/utils/misc_test.go @@ -15,84 +15,31 @@ package utils_test import ( - "context" "testing" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/br/pkg/restore" - "github.com/pingcap/tidb/br/pkg/restore/split" - "github.com/pingcap/tidb/br/pkg/utiltest" + "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/stretchr/testify/require" ) -func TestGetTSWithRetry(t *testing.T) { - t.Run("PD leader is healthy:", func(t *testing.T) { - retryTimes := -1000 - pDClient := utiltest.NewFakePDClient(nil, false, &retryTimes) - _, err := restore.GetTSWithRetry(context.Background(), pDClient) - require.NoError(t, err) - }) - - t.Run("PD leader failure:", func(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/utils/set-attempt-to-one", "1*return(true)")) - defer func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/utils/set-attempt-to-one")) - }() - retryTimes := -1000 - pDClient := utiltest.NewFakePDClient(nil, true, &retryTimes) - _, err := restore.GetTSWithRetry(context.Background(), pDClient) - require.Error(t, err) - }) - - t.Run("PD leader switch successfully", func(t *testing.T) { - retryTimes := 0 - pDClient := utiltest.NewFakePDClient(nil, true, &retryTimes) - _, err := restore.GetTSWithRetry(context.Background(), pDClient) - require.NoError(t, err) - }) +func TestTruncateTS(t *testing.T) { + keyWithTS := []byte{'1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2'} + ts := utils.TruncateTS(keyWithTS) + require.Equal(t, []byte{'1', '2', '1', '2', '1', '2', '1', '2'}, ts) + keyWithTS = []byte{'1', '2'} + ts = utils.TruncateTS(keyWithTS) + require.Equal(t, []byte{'1', '2'}, ts) } -func regionInfo(startKey, endKey string) *split.RegionInfo { - return &split.RegionInfo{ - Region: &metapb.Region{ - StartKey: []byte(startKey), - EndKey: []byte(endKey), - }, - } -} +func TestEncodeKeyPrefix(t *testing.T) { + keyPrefix := []byte{'1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2'} + encodeKey := utils.EncodeKeyPrefix(keyPrefix) + require.Equal(t, []byte{'1', '2', '1', '2', '1', '2', '1', '2', 0xff, '1', '2', '1', '2', '1', '2', '1', '2', 0xff}, encodeKey) + + keyPrefix = []byte{'1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1', '2', '1'} + encodeKey = utils.EncodeKeyPrefix(keyPrefix) + require.Equal(t, []byte{'1', '2', '1', '2', '1', '2', '1', '2', 0xff, '1', '2', '1', '2', '1', '2', '1'}, encodeKey) -func TestSplitCheckPartRegionConsistency(t *testing.T) { - var ( - startKey []byte = []byte("a") - endKey []byte = []byte("f") - err error - ) - err = split.CheckPartRegionConsistency(startKey, endKey, nil) - require.Error(t, err) - err = split.CheckPartRegionConsistency(startKey, endKey, []*split.RegionInfo{ - regionInfo("b", "c"), - }) - require.Error(t, err) - err = split.CheckPartRegionConsistency(startKey, endKey, []*split.RegionInfo{ - regionInfo("a", "c"), - regionInfo("d", "e"), - }) - require.Error(t, err) - err = split.CheckPartRegionConsistency(startKey, endKey, []*split.RegionInfo{ - regionInfo("a", "c"), - regionInfo("c", "d"), - }) - require.NoError(t, err) - err = split.CheckPartRegionConsistency(startKey, endKey, []*split.RegionInfo{ - regionInfo("a", "c"), - regionInfo("c", "d"), - regionInfo("d", "f"), - }) - require.NoError(t, err) - err = split.CheckPartRegionConsistency(startKey, endKey, []*split.RegionInfo{ - regionInfo("a", "c"), - regionInfo("c", "z"), - }) - require.NoError(t, err) + keyPrefix = []byte{'1', '2'} + encodeKey = utils.EncodeKeyPrefix(keyPrefix) + require.Equal(t, []byte{'1', '2'}, encodeKey) } diff --git a/br/pkg/restore/utils/rewrite_rule_test.go b/br/pkg/restore/utils/rewrite_rule_test.go index 5a6fd7241417c..dc8fcd68be30d 100644 --- a/br/pkg/restore/utils/rewrite_rule_test.go +++ b/br/pkg/restore/utils/rewrite_rule_test.go @@ -15,6 +15,7 @@ package utils_test import ( + "bytes" "testing" "github.com/pingcap/errors" @@ -23,6 +24,8 @@ import ( berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/br/pkg/rtree" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" @@ -280,3 +283,200 @@ func TestGetRewriteTableID(t *testing.T) { require.Equal(t, tableID, newTableID) } } + +func getNewKeyPrefix(key []byte, rewriteRules *utils.RewriteRules) kv.Key { + for _, rule := range rewriteRules.Data { + if bytes.HasPrefix(key, rule.GetOldKeyPrefix()) { + return rule.GetNewKeyPrefix() + } + } + return nil +} + +func generateRewriteTableInfos() (newTableInfo, oldTableInfo *model.TableInfo) { + newTableInfo = &model.TableInfo{ + ID: 1, + Indices: []*model.IndexInfo{ + { + ID: 1, + Name: model.NewCIStr("i1"), + }, + { + ID: 2, + Name: model.NewCIStr("i2"), + }, + }, + Partition: &model.PartitionInfo{ + Definitions: []model.PartitionDefinition{ + { + ID: 100, + Name: model.NewCIStr("p1"), + }, + { + ID: 200, + Name: model.NewCIStr("p2"), + }, + }, + }, + } + oldTableInfo = &model.TableInfo{ + ID: 2, + Indices: []*model.IndexInfo{ + { + ID: 1, + Name: model.NewCIStr("i1"), + }, + { + ID: 2, + Name: model.NewCIStr("i2"), + }, + }, + Partition: &model.PartitionInfo{ + Definitions: []model.PartitionDefinition{ + { + ID: 101, + Name: model.NewCIStr("p1"), + }, + { + ID: 201, + Name: model.NewCIStr("p2"), + }, + }, + }, + } + return newTableInfo, oldTableInfo +} + +func TestGetRewriteRules(t *testing.T) { + newTableInfo, oldTableInfo := generateRewriteTableInfos() + + { + rewriteRules := utils.GetRewriteRules(newTableInfo, oldTableInfo, 0, false) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(2), rewriteRules), tablecodec.EncodeTablePrefix(1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(101), rewriteRules), tablecodec.EncodeTablePrefix(100)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(201), rewriteRules), tablecodec.EncodeTablePrefix(200)) + } + + { + rewriteRules := utils.GetRewriteRules(newTableInfo, oldTableInfo, 0, true) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(2), rewriteRules), tablecodec.GenTableRecordPrefix(1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 1), rewriteRules), tablecodec.EncodeTableIndexPrefix(1, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 2), rewriteRules), tablecodec.EncodeTableIndexPrefix(1, 2)) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(101), rewriteRules), tablecodec.GenTableRecordPrefix(100)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(101, 1), rewriteRules), tablecodec.EncodeTableIndexPrefix(100, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(101, 2), rewriteRules), tablecodec.EncodeTableIndexPrefix(100, 2)) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(201), rewriteRules), tablecodec.GenTableRecordPrefix(200)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(201, 1), rewriteRules), tablecodec.EncodeTableIndexPrefix(200, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(201, 2), rewriteRules), tablecodec.EncodeTableIndexPrefix(200, 2)) + } +} + +func TestGetRewriteRulesMap(t *testing.T) { + newTableInfo, oldTableInfo := generateRewriteTableInfos() + + { + rewriteRules := utils.GetRewriteRulesMap(newTableInfo, oldTableInfo, 0, false) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(2), rewriteRules[2]), tablecodec.EncodeTablePrefix(1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(101), rewriteRules[101]), tablecodec.EncodeTablePrefix(100)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(201), rewriteRules[201]), tablecodec.EncodeTablePrefix(200)) + } + + { + rewriteRules := utils.GetRewriteRulesMap(newTableInfo, oldTableInfo, 0, true) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(2), rewriteRules[2]), tablecodec.GenTableRecordPrefix(1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 1), rewriteRules[2]), tablecodec.EncodeTableIndexPrefix(1, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 2), rewriteRules[2]), tablecodec.EncodeTableIndexPrefix(1, 2)) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(101), rewriteRules[101]), tablecodec.GenTableRecordPrefix(100)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(101, 1), rewriteRules[101]), tablecodec.EncodeTableIndexPrefix(100, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(101, 2), rewriteRules[101]), tablecodec.EncodeTableIndexPrefix(100, 2)) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(201), rewriteRules[201]), tablecodec.GenTableRecordPrefix(200)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(201, 1), rewriteRules[201]), tablecodec.EncodeTableIndexPrefix(200, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(201, 2), rewriteRules[201]), tablecodec.EncodeTableIndexPrefix(200, 2)) + } +} + +func TestGetRewriteRuleOfTable(t *testing.T) { + { + rewriteRules := utils.GetRewriteRuleOfTable(2, 1, 0, map[int64]int64{1: 1, 2: 2}, false) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTablePrefix(2), rewriteRules), tablecodec.EncodeTablePrefix(1)) + } + + { + rewriteRules := utils.GetRewriteRuleOfTable(2, 1, 0, map[int64]int64{1: 1, 2: 2}, true) + require.Equal(t, getNewKeyPrefix(tablecodec.GenTableRecordPrefix(2), rewriteRules), tablecodec.GenTableRecordPrefix(1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 1), rewriteRules), tablecodec.EncodeTableIndexPrefix(1, 1)) + require.Equal(t, getNewKeyPrefix(tablecodec.EncodeTableIndexPrefix(2, 2), rewriteRules), tablecodec.EncodeTableIndexPrefix(1, 2)) + } +} + +type fakeApplyFile struct { + StartKey []byte + EndKey []byte +} + +func (f fakeApplyFile) GetStartKey() []byte { + return f.StartKey +} + +func (f fakeApplyFile) GetEndKey() []byte { + return f.EndKey +} + +func rewriteKey(key kv.Key, rule *import_sstpb.RewriteRule) kv.Key { + if bytes.HasPrefix(key, rule.GetOldKeyPrefix()) { + return append(rule.GetNewKeyPrefix(), key[len(rule.GetNewKeyPrefix()):]...) + } + return nil +} + +func TestFindMatchedRewriteRule(t *testing.T) { + rewriteRules := utils.GetRewriteRuleOfTable(2, 1, 0, map[int64]int64{1: 10}, true) + { + applyFile := fakeApplyFile{ + StartKey: tablecodec.EncodeRowKeyWithHandle(2, kv.IntHandle(100)), + EndKey: tablecodec.EncodeRowKeyWithHandle(2, kv.IntHandle(200)), + } + rule := utils.FindMatchedRewriteRule(applyFile, rewriteRules) + require.Equal(t, rewriteKey(tablecodec.EncodeRowKeyWithHandle(2, kv.IntHandle(100)), rule), + tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(100))) + } + + { + applyFile := fakeApplyFile{ + StartKey: tablecodec.EncodeIndexSeekKey(2, 1, []byte("test-1")), + EndKey: tablecodec.EncodeIndexSeekKey(2, 1, []byte("test-2")), + } + rule := utils.FindMatchedRewriteRule(applyFile, rewriteRules) + require.Equal(t, rewriteKey(tablecodec.EncodeIndexSeekKey(2, 1, []byte("test-1")), rule), + tablecodec.EncodeIndexSeekKey(1, 10, []byte("test-1"))) + } + + { + applyFile := fakeApplyFile{ + StartKey: tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(100)), + EndKey: tablecodec.EncodeRowKeyWithHandle(2, kv.IntHandle(200)), + } + rule := utils.FindMatchedRewriteRule(applyFile, rewriteRules) + require.Nil(t, rule) + } + + { + applyFile := fakeApplyFile{ + StartKey: tablecodec.EncodeTablePrefix(1), + EndKey: tablecodec.EncodeTablePrefix(1), + } + rule := utils.FindMatchedRewriteRule(applyFile, rewriteRules) + require.Nil(t, rule) + } +} + +func TestGetRewriteKeyWithDifferentTable(t *testing.T) { + applyFile := fakeApplyFile{ + StartKey: tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(100)), + EndKey: tablecodec.EncodeRowKeyWithHandle(2, kv.IntHandle(200)), + } + _, _, err := utils.GetRewriteRawKeys(applyFile, nil) + require.Error(t, err) + _, _, err = utils.GetRewriteEncodedKeys(applyFile, nil) + require.Error(t, err) +} diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 050bd1748924c..9743af337113b 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -149,7 +149,7 @@ func DefineRestoreCommonFlags(flags *pflag.FlagSet) { // TODO remove experimental tag if it's stable flags.Bool(flagOnline, false, "(experimental) Whether online when restore") flags.String(flagGranularity, string(restore.CoarseGrained), "(deprecated) Whether split & scatter regions using fine-grained way during restore") - flags.Uint(flagConcurrencyPerStore, 128, "The size of thread pool on each store that executes tasks, only enabled when `--granularity=coarse-grained`") + flags.Uint(flagConcurrencyPerStore, 128, "The size of thread pool on each store that executes tasks") flags.Uint32(flagConcurrency, 128, "(deprecated) The size of thread pool on BR that executes tasks, "+ "where each task restores one SST file to TiKV") flags.Uint64(FlagMergeRegionSizeBytes, conn.DefaultMergeRegionSizeBytes, diff --git a/br/pkg/utiltest/BUILD.bazel b/br/pkg/utiltest/BUILD.bazel index 857171c570572..c8c70993f1488 100644 --- a/br/pkg/utiltest/BUILD.bazel +++ b/br/pkg/utiltest/BUILD.bazel @@ -11,12 +11,14 @@ go_library( deps = [ "//br/pkg/gluetidb/mock", "//br/pkg/mock", + "//br/pkg/pdutil", "//br/pkg/restore/split", "//br/pkg/storage", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pkg_errors//:errors", "@com_github_stretchr_testify//require", "@com_github_tikv_pd_client//:client", + "@com_github_tikv_pd_client//http", "@org_golang_google_grpc//keepalive", ], ) diff --git a/br/pkg/utiltest/fake.go b/br/pkg/utiltest/fake.go index 5bb2e339851fe..3321c141dbf76 100644 --- a/br/pkg/utiltest/fake.go +++ b/br/pkg/utiltest/fake.go @@ -17,12 +17,15 @@ package utiltest import ( "bytes" "context" + "math" "time" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pkg/errors" pd "github.com/tikv/pd/client" + pdhttp "github.com/tikv/pd/client/http" "google.golang.org/grpc/keepalive" ) @@ -31,32 +34,175 @@ var DefaultTestKeepaliveCfg = keepalive.ClientParameters{ Timeout: 10 * time.Second, } +var ( + ExpectPDCfgGeneratorsResult = map[string]any{ + "merge-schedule-limit": 0, + "leader-schedule-limit": float64(40), + "region-schedule-limit": float64(40), + "max-snapshot-count": float64(40), + "enable-location-replacement": "false", + "max-pending-peer-count": uint64(math.MaxInt32), + } + + ExistPDCfgGeneratorBefore = map[string]any{ + "merge-schedule-limit": 100, + "leader-schedule-limit": float64(100), + "region-schedule-limit": float64(100), + "max-snapshot-count": float64(100), + "enable-location-replacement": "true", + "max-pending-peer-count": 100, + } +) + +type FakePDHTTPClient struct { + pdhttp.Client + + expireSchedulers map[string]time.Time + cfgs map[string]any + + rules map[string]*pdhttp.Rule +} + +func NewFakePDHTTPClient() *FakePDHTTPClient { + return &FakePDHTTPClient{ + expireSchedulers: make(map[string]time.Time), + cfgs: make(map[string]any), + + rules: make(map[string]*pdhttp.Rule), + } +} + +func (fpdh *FakePDHTTPClient) GetScheduleConfig(_ context.Context) (map[string]any, error) { + return ExistPDCfgGeneratorBefore, nil +} + +func (fpdh *FakePDHTTPClient) GetSchedulers(_ context.Context) ([]string, error) { + schedulers := make([]string, 0, len(pdutil.Schedulers)) + for scheduler := range pdutil.Schedulers { + schedulers = append(schedulers, scheduler) + } + return schedulers, nil +} + +func (fpdh *FakePDHTTPClient) SetSchedulerDelay(_ context.Context, key string, delay int64) error { + expireTime, ok := fpdh.expireSchedulers[key] + if ok { + if time.Now().Compare(expireTime) > 0 { + return errors.Errorf("the scheduler config set is expired") + } + if delay == 0 { + delete(fpdh.expireSchedulers, key) + } + } + if !ok && delay == 0 { + return errors.Errorf("set the nonexistent scheduler") + } + expireTime = time.Now().Add(time.Second * time.Duration(delay)) + fpdh.expireSchedulers[key] = expireTime + return nil +} + +func (fpdh *FakePDHTTPClient) SetConfig(_ context.Context, config map[string]any, ttl ...float64) error { + for key, value := range config { + fpdh.cfgs[key] = value + } + return nil +} + +func (fpdh *FakePDHTTPClient) GetConfig(_ context.Context) (map[string]any, error) { + return fpdh.cfgs, nil +} + +func (fpdh *FakePDHTTPClient) GetDelaySchedulers() map[string]struct{} { + delaySchedulers := make(map[string]struct{}) + for key, t := range fpdh.expireSchedulers { + now := time.Now() + if now.Compare(t) < 0 { + delaySchedulers[key] = struct{}{} + } + } + return delaySchedulers +} + +func (fpdh *FakePDHTTPClient) GetPlacementRule(_ context.Context, groupID string, ruleID string) (*pdhttp.Rule, error) { + rule, ok := fpdh.rules[ruleID] + if !ok { + rule = &pdhttp.Rule{ + GroupID: groupID, + ID: ruleID, + } + fpdh.rules[ruleID] = rule + } + return rule, nil +} + +func (fpdh *FakePDHTTPClient) SetPlacementRule(_ context.Context, rule *pdhttp.Rule) error { + fpdh.rules[rule.ID] = rule + return nil +} + +func (fpdh *FakePDHTTPClient) DeletePlacementRule(_ context.Context, groupID string, ruleID string) error { + delete(fpdh.rules, ruleID) + return nil +} + type FakePDClient struct { pd.Client - stores []*metapb.Store + stores []*metapb.Store + regions []*pd.Region notLeader bool retryTimes *int + + peerStoreId uint64 } -func NewFakePDClient(stores []*metapb.Store, notLeader bool, retryTime *int) FakePDClient { +func NewFakePDClient(stores []*metapb.Store, notLeader bool, retryTime *int) *FakePDClient { var retryTimeInternal int if retryTime == nil { retryTime = &retryTimeInternal } - return FakePDClient{ + return &FakePDClient{ stores: stores, notLeader: notLeader, retryTimes: retryTime, + + peerStoreId: 0, } } -func (fpdc FakePDClient) GetAllStores(context.Context, ...pd.GetStoreOption) ([]*metapb.Store, error) { +func (fpdc *FakePDClient) SetRegions(regions []*pd.Region) { + fpdc.regions = regions +} + +func (fpdc *FakePDClient) GetAllStores(context.Context, ...pd.GetStoreOption) ([]*metapb.Store, error) { return append([]*metapb.Store{}, fpdc.stores...), nil } -func (fpdc FakePDClient) GetTS(ctx context.Context) (int64, int64, error) { +func (fpdc *FakePDClient) ScanRegions( + ctx context.Context, + key, endKey []byte, + limit int, + opts ...pd.GetRegionOption, +) ([]*pd.Region, error) { + regions := make([]*pd.Region, 0, len(fpdc.regions)) + fpdc.peerStoreId = fpdc.peerStoreId + 1 + peerStoreId := (fpdc.peerStoreId + 1) / 2 + for _, region := range fpdc.regions { + if len(endKey) != 0 && bytes.Compare(region.Meta.StartKey, endKey) >= 0 { + continue + } + if len(region.Meta.EndKey) != 0 && bytes.Compare(region.Meta.EndKey, key) <= 0 { + continue + } + region.Meta.Peers = []*metapb.Peer{{StoreId: peerStoreId}} + regions = append(regions, region) + } + return regions, nil +} + +func (fpdc *FakePDClient) GetTS(ctx context.Context) (int64, int64, error) { (*fpdc.retryTimes)++ if *fpdc.retryTimes >= 3 { // the mock PD leader switched successfully fpdc.notLeader = false @@ -91,6 +237,13 @@ func (f *FakeSplitClient) AppendRegion(startKey, endKey []byte) { }) } +func (f *FakeSplitClient) AppendPdRegion(region *pd.Region) { + f.regions = append(f.regions, &split.RegionInfo{ + Region: region.Meta, + Leader: region.Leader, + }) +} + func (f *FakeSplitClient) ScanRegions( ctx context.Context, startKey, endKey []byte, diff --git a/docs/design/2024-05-23-predicate-columns.md b/docs/design/2024-05-23-predicate-columns.md new file mode 100644 index 0000000000000..9598db10643ae --- /dev/null +++ b/docs/design/2024-05-23-predicate-columns.md @@ -0,0 +1,309 @@ +# Analyze Predicate Columns + +- Author(s): [Rustin Liu](http://github.com/hi-rustin) +- Discussion PR: +- Tracking Issue: + +## Table of Contents + +- [Analyze Predicate Columns](#analyze-predicate-columns) + - [Table of Contents](#table-of-contents) + - [Introduction](#introduction) + - [Motivation or Background](#motivation-or-background) + - [Detailed Design](#detailed-design) + - [ANALYZE Syntax](#analyze-syntax) + - [Tracking Predicate Columns](#tracking-predicate-columns) + - [Collect During Logical Optimization Phase](#collect-during-logical-optimization-phase) + - [Flush Predicate Columns To The System Table](#flush-predicate-columns-to-the-system-table) + - [Collection Dataflow](#collection-dataflow) + - [Using Predicate Columns](#using-predicate-columns) + - [Analysis Dataflow](#analysis-dataflow) + - [Cleanup Outdated Predicate Columns](#cleanup-outdated-predicate-columns) + - [Global Variable](#global-variable) + - [Test Design](#test-design) + - [Functional Tests](#functional-tests) + - [Compatibility Tests](#compatibility-tests) + - [Performance Tests](#performance-tests) + - [Impacts \& Risks](#impacts--risks) + - [If new predicate columns appear, they cannot be analyzed in time](#if-new-predicate-columns-appear-they-cannot-be-analyzed-in-time) + - [Use PREDICATE COLUMNS when your workload's query pattern is relatively stable](#use-predicate-columns-when-your-workloads-query-pattern-is--relatively-stable) + - [Investigation \& Alternatives](#investigation--alternatives) + - [CRDB](#crdb) + - [Summary](#summary) + - [Implementation](#implementation) + - [Redshift](#redshift) + - [Summary](#summary-1) + - [Implementation](#implementation-1) + - [Unresolved Questions](#unresolved-questions) + +## Introduction + +This document describes the design of the feature that allows TiDB to analyze only the predicate columns when executing the `ANALYZE` statement. This feature is designed to reduce the cost of `ANALYZE` and improve the efficiency of analyzing large tables. + +## Motivation or Background + +The ANALYZE statement would collect the statistics of all columns currently. If the table is big and wide, executing `ANALYZE` would consume lots of time, memory, and CPU. See [#27358](https://github.com/pingcap/tidb/issues/27358) for details. +However, only the statistics of some columns are used in creating query plans, while the statistics of others are not. Predicate columns are those columns whose statistics are used in query plans, usually in where conditions, join conditions, and so on. If ANALYZE only collects statistics for predicate columns and indexed columns (statistics of indexed columns are important for index selection), the cost of ANALYZE can be reduced. + +## Detailed Design + +### ANALYZE Syntax + +```sql +ANALYZE TABLE tbl_name PREDICATE COLUMNS; +``` + +Using this syntax, TiDB will only analyze columns that appear in the predicate of the query. + +Compare with other syntaxes: + +| Analyze Statement | Explain | +| ----------------------------------- | ---------------------------------------------------------------- | +| ANALYZE TABLE t; | It will analyze with default options. (Usually all columns) | +| ANALYZE TABLE t ALL COLUMNS; | It will analyze all columns. | +| ANALYZE TABLE t COLUMNS col1, col2; | It will only analyze col1 and col2. | +| ANALYZE TABLE t PREDICATE COLUMNS; | It will only analyze columns that exist in the previous queries. | + +### Tracking Predicate Columns + +#### Collect During Logical Optimization Phase + +As all the predicates need to be parsed for each query, we need to do it during the logical optimization phase. +It requires a new logical optimization rule, and we can use it to go through the whole plan tree and try to find all the predicate columns. + +We will consider these columns to be the predicate columns: + +- PushDown Conditions from DataSource +- Access Conditions from DataSource +- Conditions from Select +- GroupBy Items from Aggregation +- PartitionBy from the Window function +- Equal Conditions, Left Conditions, Right Conditions and Other Conditions from JOIN +- Correlated Columns from Apply +- SortBy Items from Sort +- SortBy Items from TopN +- Columns from the CTE if distinct specified + +After we get all predicate columns from the plan tree, we can store them in memory. The reason for storing them in memory is that we don't want to slow down the optimization process by sending the request to TiKV. +Additionally, we want to record when this column was used, we also need to record the timestamp. + +It is a map from `TableItemID` to `time.Time`: + +```go +type TableItemID struct { + TableID int64 + ID int64 + IsIndex bool + IsSyncLoadFailed bool +} + +// StatsUsage maps (tableID, columnID) to the last time when the column stats are used(needed). +// All methods of it are thread-safe. +type StatsUsage struct { + usage map[model.TableItemID]time.Time + lock sync.RWMutex +} +``` + +#### Flush Predicate Columns To The System Table + +We use a new system table `mysql.column_stats_usage` to store predicate columns. + +```sql +CREATE TABLE IF NOT EXISTS mysql.column_stats_usage ( + table_id BIGINT(64) NOT NULL, + column_id BIGINT(64) NOT NULL, + last_used_at TIMESTAMP, + last_analyzed_at TIMESTAMP, + PRIMARY KEY (table_id, column_id) CLUSTERED +); +``` + +The detailed explanation: + +| Column Name | Description | +| ---------------- | ---------------------------------------------------- | +| table_id | The physical table ID. | +| column_id | The column ID from schema information. | +| last_used_at | The timestamp when the column statistics were used. | +| last_analyzed_at | The timestamp at when the column stats were updated. | + +After we collect all predicate columns in the memory, we can use a background worker to flush them from the memory to TiKV. The pseudo-code looks like this: + +```go +func (do *Domain) updateStatsWorker() { + dumpColStatsUsageTicker := time.NewTicker(100 * lease) + for { + select { + case <-dumpColStatsUsageTicker.C: + statsHandle.DumpColStatsUsageToKV() + } + } +} + +func (s *statsUsageImpl) DumpColStatsUsageToKV() error { + colMap := getAllPredicateColumns + for col, time := range colMap { + StoreToSystemTable(col, time) + } +} +``` + +As illustrated, a ticker triggers the flush operation. This operation retrieves all predicate columns from memory and stores them in the system table. The ticker interval is set to 100 times the lease, where the lease is derived from the [statistics lease](https://docs.pingcap.com/tidb/stable/tidb-configuration-file#stats-lease). +So it is adjustable according to the [statistics lease](https://docs.pingcap.com/tidb/stable/tidb-configuration-file#stats-lease). + +#### Collection Dataflow + +![Dataflow](./imgs/predicate-columns-collect.png) + +### Using Predicate Columns + +We can use the predicate columns in both automatic statistics collection and manual statistics collection. + +Whether the statistics collection is triggered automatically or manually, it reads the predicate columns from the system table and only analyzes these columns, provided that the PREDICATE COLUMNS have already been configured in the analyze options. + +To comprehend the application of predicate columns in automatic statistics collection, it's essential to understand how TiDB persists the analyze options. + +In TiDB, we use a system table to store the analyze options, called `mysql.analyze_options`: + +```sql +CREATE TABLE IF NOT EXISTS mysql.analyze_options ( + table_id BIGINT(64) NOT NULL, + sample_num BIGINT(64) NOT NULL DEFAULT 0, + sample_rate DOUBLE NOT NULL DEFAULT -1, + buckets BIGINT(64) NOT NULL DEFAULT 0, + topn BIGINT(64) NOT NULL DEFAULT -1, + column_choice enum('DEFAULT','ALL','PREDICATE','LIST') NOT NULL DEFAULT 'DEFAULT', + column_ids TEXT(19372), + PRIMARY KEY (table_id) CLUSTERED +); +``` + +We can focus on the `column_choice` column, which has three different column options in the analyze statement. The corresponding relations are as follows: + +| Analyze Statement | column_choice | column_ids | mysql.column_stats_usage | Explain | +| ---------------------------------------- | ------------- | ---------- | ------------------------------------------------------------------------ | ------------------------------------------------------------------------ | +| ANALYZE TABLE t; | DEFAULT(ALL) | None | None | It will analyze all analyzable columns from the table. | +| ANALYZE TABLE t ALL COLUMNS; | ALL | None | None | It will analyze all columns from the table. | +| ANALYZE TABLE t LIST COLUMNS col1, col2; | LIST | col1, col2 | None | It will only analyze col1 and col2. | +| ANALYZE TABLE t PREDICATE COLUMNS; | PREDICATE | None | All predicate columns were collected before in mysql.column_stats_usage. | It will only analyze columns that exist in the mysql.column_stats_usage. | + +As you can see, we pick PREDICATE as the column_choice for the ANALYZE TABLE t PREDICATE COLUMNS statement. At the same time, we now consider DEFAULT to be ALL, but to support predicate columns during auto-analyze, we need to change the definition of DEFAULT. + +| Predicate Column Feature Status | Predicate Columns in `mysql.column_stats_usage` | Meaning | +| ------------------------------- | ----------------------------------------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| Enabled | Present | Use those predicate and indexed columns to analyze the table. | +| Enabled | Absent | Only analyze indexed columns of the table. **If there are no indexes present, we will bypass the analysis of this table and directly set the modify_count to 0** | +| Disabled | - | Analyze all columns of the table | + +After we change the definition of DEFAULT, we can use the predicate columns to analyze the table during auto-analyze. + +#### Analysis Dataflow + +![Dataflow](./imgs/auto-analyze-predicate-columns.png) + +### Cleanup Outdated Predicate Columns + +Users may have made schema changes, requiring the removal of non-existent columns from the `mysql.column_stats_usage` table. + +Before initiating the analyze process, we can first retrieve all predicate columns, compare them with the current schema, and remove any columns that no longer exist from the `mysql.column_stats_usage` table. + +### Global Variable + +In the experimental implementation, we introduce a new global variable `tidb_enable_column_tracking` to control whether to use predicate columns in the analyze process. + +But because we decided to track all columns by default, so it becomes unnecessary to use this variable. We will mark it deprecated and remove it in the future. + +In this feature, we introduce a new global variable `tidb_analyze_default_column_choice` to control whether to use predicate columns or all columns in the analyze process. + +Users can set this variable to `ALL` or `PREDICATE` to analyze all columns or only predicate columns. The default value will be `PREDICATE` after this feature is fully implemented. + +```sql +SET GLOBAL tidb_analyze_default_column_choice = 'PREDICATE'; + +SET GLOBAL tidb_analyze_default_column_choice = 'ALL'; +``` + +| Value | Description | +| --------- | -------------------------------------------------------------- | +| PREDICATE | Use predicate columns in the analyze process. | +| ALL | Analyze all columns (**But still collect predicate columns**). | + +We continue to collect predicate columns even when the feature is disabled. This ensures that we can promptly catch up with the latest predicate columns when the feature is re-enabled. + +## Test Design + +This feature requires a series of tests to ensure its functionality and compatibility. + +### Functional Tests + +1. Test that the predicate columns are correctly collected and stored in the system table. +2. Test that the predicate columns are correctly used in the analyze process. +3. Test that the query plan is correct after the predicate columns are used in the analyze process. +4. Test that the predicate columns are correctly cleaned up when they are no longer exist. + +### Compatibility Tests + +1. Test that the feature is compatible with the auto-analyze feature. +2. Test that the feature is compatible with the rolling upgrade. + +### Performance Tests + +Test that the query plan is optimized after the predicate columns are used in the analyze process. + +## Impacts & Risks + +### If new predicate columns appear, they cannot be analyzed in time + +If a table has some new queries that use some new predicate columns, but we have just finished an auto-analysis for this table. Then for a long time, the optimizer cannot get the statistics for the newly used columns until we have enough modification to trigger the auto-analysis again. +**This problem is more obvious during cluster creation or POC.** But by default, we always collect statistics for all indexes, so we consider this problem to be acceptable. + +### Use PREDICATE COLUMNS when your workload's query pattern is relatively stable + +When the query pattern is variable, with different columns frequently being used as predicates, using PREDICATE COLUMNS might temporarily result in stale statistics. Stale statistics can lead to suboptimal query runtime plans and long runtimes. + +## Investigation & Alternatives + +### CRDB + +#### Summary + +By default, CockroachDB automatically generates table statistics when tables are created, and as they are updated. It does this using a background job that automatically determines which columns to get statistics on — specifically, it chooses: + +- Columns that are part of the primary key or an index (in other words, all indexed columns). +- Up to 100 non-indexed columns. + +#### Implementation + +If the column list is not specified in the analyze SQL, the default column list will be used to generate statistics. +To determine a useful set of default column statistics, they rely on information provided by the schema. + +1. The presence of an index on a particular set of columns indicates that the workload likely contains queries that involve those columns (e.g., for filters), and it would be useful to have statistics on prefixes of those columns. For example, if a table abc contains indexes on (a ASC, b ASC) and (b ASC, c ASC), we will collect statistics on a, {a, b}, b, and {b, c}. (But if ```multiColEnabled``` is false, they will only collect stats on a and b). +2. Columns in partial index predicate expressions are also likely to appear in query filters, so stats are collected for those columns as well. +3. They only collect histograms for index columns, plus any other boolean or enum columns (where the "histogram" is tiny). + +See more: + +### Redshift + +#### Summary + +Amazon Redshift ANALYZE command can optionally collect information only about columns used in previous queries as part of a filter, join condition or a GROUP BY clause, and columns that are part of the distribution or sort keys (predicate columns). There’s a recently introduced option for the ANALYZE command that only analyzes predicate columns: + +```sql +ANALYZE PREDICATE COLUMNS; +``` + +#### Implementation + +When you run ANALYZE with the PREDICATE COLUMNS clause, the analyze operation includes only columns that meet the following criteria: + +- The column is marked as a predicate column. +- The column is a distribution key. +- The column is part of a sort key. + +If none of a table's columns are marked as predicates, ANALYZE includes all of the columns, even when PREDICATE COLUMNS is specified. If no columns are marked as predicate columns, it might be because the table has not yet been queried. + +## Unresolved Questions + +None diff --git a/docs/design/imgs/auto-analyze-predicate-columns.png b/docs/design/imgs/auto-analyze-predicate-columns.png new file mode 100644 index 0000000000000..9af99646e3e4e Binary files /dev/null and b/docs/design/imgs/auto-analyze-predicate-columns.png differ diff --git a/docs/design/imgs/predicate-columns-collect.png b/docs/design/imgs/predicate-columns-collect.png new file mode 100644 index 0000000000000..29d544caa8118 Binary files /dev/null and b/docs/design/imgs/predicate-columns-collect.png differ diff --git a/go.mod b/go.mod index 236bc403f99b9..634438870e493 100644 --- a/go.mod +++ b/go.mod @@ -107,7 +107,7 @@ require ( github.com/tdakkota/asciicheck v0.2.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tidwall/btree v1.7.0 - github.com/tikv/client-go/v2 v2.0.8-0.20240604045705-156cebc2defa + github.com/tikv/client-go/v2 v2.0.8-0.20240614064455-ac8fa1d73a0c github.com/tikv/pd/client v0.0.0-20240603082825-a929a546a790 github.com/timakin/bodyclose v0.0.0-20240125160201-f835fa56326a github.com/twmb/murmur3 v1.1.6 diff --git a/go.sum b/go.sum index 349655360bb2d..be7b54c7e3ea9 100644 --- a/go.sum +++ b/go.sum @@ -846,8 +846,8 @@ github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tidwall/btree v1.7.0 h1:L1fkJH/AuEh5zBnnBbmTwQ5Lt+bRJ5A8EWecslvo9iI= github.com/tidwall/btree v1.7.0/go.mod h1:twD9XRA5jj9VUQGELzDO4HPQTNJsoWWfYEL+EUQ2cKY= -github.com/tikv/client-go/v2 v2.0.8-0.20240604045705-156cebc2defa h1:9GSe3tYLJlhGx78eWBht0Yp/Q5LY2zZ1ExDOP/mGzwo= -github.com/tikv/client-go/v2 v2.0.8-0.20240604045705-156cebc2defa/go.mod h1:GHzfy/lS+Gr9emV8OwU+k4kXCB3/8H51DZBFDTeyE84= +github.com/tikv/client-go/v2 v2.0.8-0.20240614064455-ac8fa1d73a0c h1:lmC//5VZCLMYAzCVazTnopQoififZfXqnaIZqTAXL9Q= +github.com/tikv/client-go/v2 v2.0.8-0.20240614064455-ac8fa1d73a0c/go.mod h1:GHzfy/lS+Gr9emV8OwU+k4kXCB3/8H51DZBFDTeyE84= github.com/tikv/pd/client v0.0.0-20240603082825-a929a546a790 h1:bGmvWcMkbOlVgWpsXza2gu18Ud2dEyTz60UU2oEUSoA= github.com/tikv/pd/client v0.0.0-20240603082825-a929a546a790/go.mod h1:kNRekhwXqjTjNHy+kPmbZvsMmvl42zOj/UW5IIG+nP0= github.com/timakin/bodyclose v0.0.0-20240125160201-f835fa56326a h1:A6uKudFIfAEpoPdaal3aSqGxBzLyU8TqyXImLwo6dIo= diff --git a/pkg/bindinfo/session_handle_test.go b/pkg/bindinfo/session_handle_test.go index ae653ef17297a..5d5bc3d7bb1fa 100644 --- a/pkg/bindinfo/session_handle_test.go +++ b/pkg/bindinfo/session_handle_test.go @@ -395,9 +395,13 @@ func TestIssue53834(t *testing.T) { defer func() { tk.MustExec(fmt.Sprintf(`set global tidb_mem_oom_action='%v'`, oomAction)) }() + tk.MustExec(`set global tidb_mem_oom_action='cancel'`) + err := tk.ExecToErr(`replace into t select /*+ memory_quota(1 mb) */ * from t`) + require.ErrorContains(t, err, "cancelled due to exceeding the allowed memory limit") + tk.MustExec(`create binding using replace into t select /*+ memory_quota(1 mb) */ * from t`) - err := tk.ExecToErr(`replace into t select * from t`) + err = tk.ExecToErr(`replace into t select * from t`) require.ErrorContains(t, err, "cancelled due to exceeding the allowed memory limit") } diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index b683bf999ff87..0895c38bca1ae 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -1681,7 +1681,8 @@ func getLocalWriterConfig(indexCnt, writerCnt int) *backend.LocalWriterConfig { return writerCfg } - availMem := memRoot.MaxMemoryQuota() - memRoot.CurrentUsage() + // leave some room for objects overhead + availMem := memRoot.MaxMemoryQuota() - memRoot.CurrentUsage() - int64(10*size.MB) memLimitPerWriter := availMem / int64(indexCnt) / int64(writerCnt) memLimitPerWriter = min(memLimitPerWriter, litconfig.DefaultLocalWriterMemCacheSize) writerCfg.Local.MemCacheSize = memLimitPerWriter diff --git a/pkg/ddl/ingest/engine.go b/pkg/ddl/ingest/engine.go index 2367210508598..7bbfdf15d56ef 100644 --- a/pkg/ddl/ingest/engine.go +++ b/pkg/ddl/ingest/engine.go @@ -230,7 +230,7 @@ func (ei *engineInfo) newWriterContext(workerID int, writerCfg *backend.LocalWri } // Cache the local writer. ei.writerCache.Store(workerID, lWrite) - ei.memRoot.Consume(writerCfg.Local.MemCacheSize) + ei.memRoot.ConsumeWithTag(encodeBackendTag(ei.jobID), writerCfg.Local.MemCacheSize) } wc := &writerContext{ ctx: ei.ctx, @@ -250,7 +250,6 @@ func (ei *engineInfo) closeWriters() error { firstErr = err } } - ei.memRoot.Release(int64(ei.litCfg.TikvImporter.LocalWriterMemCacheSize)) } ei.writerCache.Delete(wid) ei.memRoot.Release(structSizeWriterCtx) diff --git a/pkg/ddl/ingest/mem_root.go b/pkg/ddl/ingest/mem_root.go index e254620a69704..5fa69eda5a916 100644 --- a/pkg/ddl/ingest/mem_root.go +++ b/pkg/ddl/ingest/mem_root.go @@ -25,6 +25,9 @@ type MemRoot interface { Consume(size int64) Release(size int64) CheckConsume(size int64) bool + // ConsumeWithTag consumes memory with a tag. The main difference between + // ConsumeWithTag and Consume is that if the memory is updated afterward, caller + // can use ReleaseWithTag then ConsumeWithTag to update the memory usage. ConsumeWithTag(tag string, size int64) ReleaseWithTag(tag string) diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 313c937e6adb9..e28380be5aae2 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -683,7 +683,7 @@ func (b *executorBuilder) buildCleanupIndex(v *plannercore.CleanupIndex) exec.Ex sessCtx := e.Ctx().GetSessionVars().StmtCtx e.handleCols = buildHandleColsForExec(sessCtx, tblInfo, e.columns) if e.index.Meta().Global { - e.columns = append(e.columns, model.NewExtraPartitionIDColInfo()) + e.columns = append(e.columns, model.NewExtraPhysTblIDColInfo()) } return e } @@ -5133,7 +5133,7 @@ func NewRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model } defVal := func(i int, chk *chunk.Chunk) error { if reqCols[i].ID < 0 { - // model.ExtraHandleID, ExtraPidColID, ExtraPhysTblID... etc + // model.ExtraHandleID, ExtraPhysTblID... etc // Don't set the default value for that column. chk.AppendNull(i) return nil diff --git a/pkg/executor/delete.go b/pkg/executor/delete.go index 2af45fb344d0a..6ec0b696a9c0b 100644 --- a/pkg/executor/delete.go +++ b/pkg/executor/delete.go @@ -98,6 +98,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { batchDelete := e.Ctx().GetSessionVars().BatchDelete && !e.Ctx().GetSessionVars().InTxn() && variable.EnableBatchDML.Load() && batchDMLSize > 0 fields := exec.RetTypes(e.Children(0)) + datumRow := make([]types.Datum, 0, len(fields)) chk := exec.TryNewCacheChunk(e.Children(0)) columns := e.Children(0).Schema().Columns if len(columns) != len(fields) { @@ -127,10 +128,8 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { } rowCount = 0 } - - datumRow := make([]types.Datum, 0, len(fields)) for i, field := range fields { - if columns[i].ID == model.ExtraPidColID || columns[i].ID == model.ExtraPhysTblID { + if columns[i].ID == model.ExtraPhysTblID { continue } @@ -143,6 +142,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { return err } rowCount++ + datumRow = datumRow[:0] } chk = chunk.Renew(chk, e.MaxChunkSize()) if txn, _ := e.Ctx().Txn(false); txn != nil { @@ -171,6 +171,7 @@ func (e *DeleteExec) doBatchDelete(ctx context.Context) error { func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []plannercore.TblColPosInfo, joinedRow []types.Datum) error { // iterate all the joined tables, and got the corresponding rows in joinedRow. + var totalMemDelta int64 for _, info := range colPosInfos { if unmatchedOuterRow(info, joinedRow) { continue @@ -195,8 +196,9 @@ func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []p memDelta += types.EstimatedMemUsage(joinedRow, 1) memDelta += int64(handle.ExtraMemSize()) } - e.memTracker.Consume(memDelta) + totalMemDelta += memDelta } + e.memTracker.Consume(totalMemDelta) return nil } diff --git a/pkg/executor/distsql.go b/pkg/executor/distsql.go index 50eb917cf225a..146dfbbd3c3bd 100644 --- a/pkg/executor/distsql.go +++ b/pkg/executor/distsql.go @@ -581,14 +581,13 @@ func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize in func (e *IndexLookUpExecutor) needPartitionHandle(tp getHandleType) (bool, error) { var col *expression.Column - var needPartitionHandle, hasExtraCol bool + var needPartitionHandle bool if tp == getHandleFromIndex { cols := e.idxPlans[0].Schema().Columns outputOffsets := e.dagPB.OutputOffsets col = cols[outputOffsets[len(outputOffsets)-1]] // For indexScan, need partitionHandle when global index or keepOrder with partitionTable needPartitionHandle = e.index.Global || e.partitionTableMode && e.keepOrder - hasExtraCol = col.ID == model.ExtraPhysTblID || col.ID == model.ExtraPidColID } else { cols := e.tblPlans[0].Schema().Columns outputOffsets := e.tableRequest.OutputOffsets @@ -596,9 +595,8 @@ func (e *IndexLookUpExecutor) needPartitionHandle(tp getHandleType) (bool, error // For TableScan, need partitionHandle in `indexOrder` when e.keepOrder == true or execute `admin check [table|index]` with global index needPartitionHandle = ((e.index.Global || e.partitionTableMode) && e.keepOrder) || (e.index.Global && e.checkIndexValue != nil) - // no ExtraPidColID here, because TableScan shouldn't contain them. - hasExtraCol = col.ID == model.ExtraPhysTblID } + hasExtraCol := col.ID == model.ExtraPhysTblID // There will be two needPartitionHandle != hasExtraCol situations. // Only `needPartitionHandle` == true and `hasExtraCol` == false are not allowed. diff --git a/pkg/executor/index_merge_reader.go b/pkg/executor/index_merge_reader.go index 79b696e8966cd..856b86f115521 100644 --- a/pkg/executor/index_merge_reader.go +++ b/pkg/executor/index_merge_reader.go @@ -628,7 +628,6 @@ func (w *partialTableWorker) needPartitionHandle() (bool, error) { col := cols[outputOffsets[len(outputOffsets)-1]] needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0 - // no ExtraPidColID here, because a clustered index couldn't be a global index. hasExtraCol := col.ID == model.ExtraPhysTblID // There will be two needPartitionHandle != hasExtraCol situations. @@ -1698,7 +1697,7 @@ func syncErr(ctx context.Context, finished <-chan struct{}, errCh chan<- *indexM } // needPartitionHandle indicates whether we need create a partitionHandle or not. -// If the schema from planner part contains ExtraPidColID or ExtraPhysTblID, +// If the schema from planner part contains ExtraPhysTblID, // we need create a partitionHandle, otherwise create a normal handle. // In TableRowIDScan, the partitionHandle will be used to create key ranges. func (w *partialIndexWorker) needPartitionHandle() (bool, error) { @@ -1706,8 +1705,9 @@ func (w *partialIndexWorker) needPartitionHandle() (bool, error) { outputOffsets := w.dagPB.OutputOffsets col := cols[outputOffsets[len(outputOffsets)-1]] - needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0 - hasExtraCol := col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID + is := w.plan[0].(*plannercore.PhysicalIndexScan) + needPartitionHandle := w.partitionTableMode && len(w.byItems) > 0 || is.Index.Global + hasExtraCol := col.ID == model.ExtraPhysTblID // There will be two needPartitionHandle != hasExtraCol situations. // Only `needPartitionHandle` == true and `hasExtraCol` == false are not allowed. @@ -1715,7 +1715,7 @@ func (w *partialIndexWorker) needPartitionHandle() (bool, error) { if needPartitionHandle && !hasExtraCol { return needPartitionHandle, errors.Errorf("Internal error, needPartitionHandle != ret") } - return needPartitionHandle || (col.ID == model.ExtraPidColID), nil + return needPartitionHandle, nil } func (w *partialIndexWorker) fetchHandles( diff --git a/pkg/executor/insert_common.go b/pkg/executor/insert_common.go index da1d9bd5f6703..185882b3065c7 100644 --- a/pkg/executor/insert_common.go +++ b/pkg/executor/insert_common.go @@ -474,6 +474,7 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error { } chkMemUsage := chk.MemoryUsage() memTracker.Consume(chkMemUsage) + var totalMemDelta int64 for innerChunkRow := iter.Begin(); innerChunkRow != iter.End(); innerChunkRow = iter.Next() { innerRow := innerChunkRow.GetDatumRow(fields) e.rowCount++ @@ -486,21 +487,21 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error { if batchInsert && e.rowCount%uint64(batchSize) == 0 { memUsageOfRows = types.EstimatedMemUsage(rows[0], len(rows)) memUsageOfExtraCols = types.EstimatedMemUsage(extraColsInSel[0], len(extraColsInSel)) - memTracker.Consume(memUsageOfRows + memUsageOfExtraCols) + totalMemDelta += memUsageOfRows + memUsageOfExtraCols e.Ctx().GetSessionVars().CurrInsertBatchExtraCols = extraColsInSel if err = base.exec(ctx, rows); err != nil { return err } rows = rows[:0] extraColsInSel = extraColsInSel[:0] - memTracker.Consume(-memUsageOfRows) - memTracker.Consume(-memUsageOfExtraCols) + totalMemDelta += -memUsageOfRows - memUsageOfExtraCols memUsageOfRows = 0 if err = e.doBatchInsert(ctx); err != nil { return err } } } + memTracker.Consume(totalMemDelta) if len(rows) != 0 { memUsageOfRows = types.EstimatedMemUsage(rows[0], len(rows)) @@ -514,9 +515,7 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error { } rows = rows[:0] extraColsInSel = extraColsInSel[:0] - memTracker.Consume(-memUsageOfRows) - memTracker.Consume(-memUsageOfExtraCols) - memTracker.Consume(-chkMemUsage) + memTracker.Consume(-memUsageOfRows - memUsageOfExtraCols - chkMemUsage) } return nil } diff --git a/pkg/executor/internal/vecgroupchecker/BUILD.bazel b/pkg/executor/internal/vecgroupchecker/BUILD.bazel index ba5fce361b700..3a143a8cf4ace 100644 --- a/pkg/executor/internal/vecgroupchecker/BUILD.bazel +++ b/pkg/executor/internal/vecgroupchecker/BUILD.bazel @@ -22,7 +22,7 @@ go_test( ], embed = [":vecgroupchecker"], flaky = True, - shard_count = 3, + shard_count = 4, deps = [ "//pkg/config", "//pkg/expression", diff --git a/pkg/executor/internal/vecgroupchecker/vec_group_checker.go b/pkg/executor/internal/vecgroupchecker/vec_group_checker.go index 056ee8a478032..abb070420254e 100644 --- a/pkg/executor/internal/vecgroupchecker/vec_group_checker.go +++ b/pkg/executor/internal/vecgroupchecker/vec_group_checker.go @@ -496,6 +496,7 @@ func (e *VecGroupChecker) IsExhausted() bool { func (e *VecGroupChecker) Reset() { if e.groupOffset != nil { e.groupOffset = e.groupOffset[:0] + e.groupCount = 0 } if e.sameGroup != nil { e.sameGroup = e.sameGroup[:0] diff --git a/pkg/executor/internal/vecgroupchecker/vec_group_checker_test.go b/pkg/executor/internal/vecgroupchecker/vec_group_checker_test.go index cbb2ec240b395..df9559eaea3c1 100644 --- a/pkg/executor/internal/vecgroupchecker/vec_group_checker_test.go +++ b/pkg/executor/internal/vecgroupchecker/vec_group_checker_test.go @@ -268,3 +268,13 @@ func TestVecGroupChecker(t *testing.T) { require.Equal(t, e, 3) require.True(t, groupChecker.IsExhausted()) } + +func TestIssue53867(t *testing.T) { + checker := NewVecGroupChecker(nil, true, nil) + checker.groupOffset = make([]int, 20) + checker.nextGroupID = 10 + checker.groupCount = 15 + require.False(t, checker.IsExhausted()) + checker.Reset() + require.True(t, checker.IsExhausted()) +} diff --git a/pkg/executor/partition_table_test.go b/pkg/executor/partition_table_test.go index 6e6f8335b1d36..52ff2be31f002 100644 --- a/pkg/executor/partition_table_test.go +++ b/pkg/executor/partition_table_test.go @@ -2407,3 +2407,32 @@ func TestIssue31024(t *testing.T) { tk2.MustExec("rollback") } + +func TestGlobalIndexWithSelectLock(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("set tidb_enable_global_index = true") + tk1.MustExec("use test") + tk1.MustExec("create table t(a int, b int, unique index(b), primary key(a)) partition by hash(a) partitions 5;") + tk1.MustExec("insert into t values (1,1),(2,2),(3,3),(4,4),(5,5);") + tk1.MustExec("begin") + tk1.MustExec("select * from t use index(b) where b = 2 order by b limit 1 for update;") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + ch := make(chan int, 10) + go func() { + // Check the key is locked. + tk2.MustExec("update t set b = 6 where b = 2") + ch <- 1 + }() + + time.Sleep(50 * time.Millisecond) + ch <- 0 + tk1.MustExec("commit") + + require.Equal(t, <-ch, 0) + require.Equal(t, <-ch, 1) +} diff --git a/pkg/executor/update.go b/pkg/executor/update.go index 4a50ccd75b909..bda1be8422ea2 100644 --- a/pkg/executor/update.go +++ b/pkg/executor/update.go @@ -124,6 +124,7 @@ func (e *UpdateExec) merge(row, newData []types.Datum, mergeGenerated bool) erro } var mergedData []types.Datum // merge updates from and into mergedRowData + var totalMemDelta int64 for i, content := range e.tblColPosInfos { if !e.multiUpdateOnSameTable[content.TblID] { // No need to merge if not multi-updated @@ -165,8 +166,9 @@ func (e *UpdateExec) merge(row, newData []types.Datum, mergeGenerated bool) erro memDelta := e.mergedRowData[content.TblID].Set(handle, mergedData) memDelta += types.EstimatedMemUsage(mergedData, 1) + int64(handle.ExtraMemSize()) - e.memTracker.Consume(memDelta) + totalMemDelta += memDelta } + e.memTracker.Consume(totalMemDelta) return nil } @@ -176,6 +178,10 @@ func (e *UpdateExec) exec(ctx context.Context, _ *expression.Schema, row, newDat for i, flag := range e.assignFlag { bAssignFlag[i] = flag >= 0 } + + var totalMemDelta int64 + defer func() { e.memTracker.Consume(totalMemDelta) }() + for i, content := range e.tblColPosInfos { if !e.tableUpdatable[i] { // If there's nothing to update, we can just skip current row @@ -206,7 +212,7 @@ func (e *UpdateExec) exec(ctx context.Context, _ *expression.Schema, row, newDat if !exist { memDelta += int64(handle.ExtraMemSize()) } - e.memTracker.Consume(memDelta) + totalMemDelta += memDelta continue } diff --git a/pkg/expression/explain.go b/pkg/expression/explain.go index 0df45007d1346..2162e84144448 100644 --- a/pkg/expression/explain.go +++ b/pkg/expression/explain.go @@ -37,11 +37,11 @@ func (expr *ScalarFunction) explainInfo(ctx EvalContext, normalized bool) string intest.Assert(normalized || ctx != nil) var buffer bytes.Buffer fmt.Fprintf(&buffer, "%s(", expr.FuncName.L) - // convert `in(_tidb_pid, -1)` to `in(_tidb_pid, dual)` whether normalized equals to true or false. + // convert `in(_tidb_tid, -1)` to `in(_tidb_tid, dual)` whether normalized equals to true or false. if expr.FuncName.L == ast.In { args := expr.GetArgs() - if len(args) == 2 && args[0].ExplainNormalizedInfo() == model.ExtraPartitionIdName.L && args[1].(*Constant).Value.GetInt64() == -1 { - buffer.WriteString(model.ExtraPartitionIdName.L + ", dual)") + if len(args) == 2 && strings.HasSuffix(args[0].ExplainNormalizedInfo(), model.ExtraPhysTblIdName.L) && args[1].(*Constant).Value.GetInt64() == -1 { + buffer.WriteString(args[0].ExplainNormalizedInfo() + ", dual)") return buffer.String() } } diff --git a/pkg/kv/interface_mock_test.go b/pkg/kv/interface_mock_test.go index 9ea952cfb750f..54345af455c29 100644 --- a/pkg/kv/interface_mock_test.go +++ b/pkg/kv/interface_mock_test.go @@ -172,6 +172,8 @@ func (t *mockTxn) SetMemoryFootprintChangeHook(func(uint64)) { } +func (t *mockTxn) MemHookSet() bool { return false } + func (t *mockTxn) Mem() uint64 { return 0 } diff --git a/pkg/kv/kv.go b/pkg/kv/kv.go index aa2b76b952394..8dc08c8adc1fb 100644 --- a/pkg/kv/kv.go +++ b/pkg/kv/kv.go @@ -222,6 +222,8 @@ type Transaction interface { Mem() uint64 // SetMemoryFootprintChangeHook sets the hook that will be called when the memory footprint changes. SetMemoryFootprintChangeHook(func(uint64)) + // MemHookSet returns whether the memory footprint change hook is set. + MemHookSet() bool // Len returns the number of entries in the DB. Len() int // Commit commits the transaction operations to KV store. diff --git a/pkg/lightning/backend/local/BUILD.bazel b/pkg/lightning/backend/local/BUILD.bazel index 0033107aa3668..71c064fc2f845 100644 --- a/pkg/lightning/backend/local/BUILD.bazel +++ b/pkg/lightning/backend/local/BUILD.bazel @@ -157,6 +157,7 @@ go_test( "//pkg/util/codec", "//pkg/util/engine", "//pkg/util/hack", + "//pkg/util/mathutil", "//pkg/util/mock", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//objstorage/objstorageprovider", @@ -171,10 +172,12 @@ go_test( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/errorpb", "@com_github_pingcap_kvproto//pkg/import_sstpb", + "@com_github_pingcap_kvproto//pkg/keyspacepb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_tipb//go-tipb", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", + "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_pd_client//:client", "@com_github_tikv_pd_client//errs", "@com_github_tikv_pd_client//http", diff --git a/pkg/lightning/backend/local/engine.go b/pkg/lightning/backend/local/engine.go index 27bba1f826071..9374b2ade74ff 100644 --- a/pkg/lightning/backend/local/engine.go +++ b/pkg/lightning/backend/local/engine.go @@ -27,6 +27,7 @@ import ( "slices" "sync" "time" + "unsafe" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/objstorage/objstorageprovider" @@ -262,6 +263,8 @@ func (e *Engine) unlock() { e.mutex.Unlock() } +var sizeOfKVPair = int64(unsafe.Sizeof(common.KvPair{})) + // TotalMemorySize returns the total memory size of the engine. func (e *Engine) TotalMemorySize() int64 { var memSize int64 @@ -272,6 +275,9 @@ func (e *Engine) TotalMemorySize() int64 { memSize += w.kvBuffer.TotalSize() w.Unlock() } + w.Lock() + memSize += sizeOfKVPair * int64(cap(w.writeBatch)) + w.Unlock() return true }) return memSize diff --git a/pkg/lightning/backend/local/engine_mgr.go b/pkg/lightning/backend/local/engine_mgr.go index 6dd812d794488..28b6107a3d3e5 100644 --- a/pkg/lightning/backend/local/engine_mgr.go +++ b/pkg/lightning/backend/local/engine_mgr.go @@ -69,6 +69,8 @@ type engineManager struct { logger log.Logger } +var inMemTest = false + func newEngineManager(config BackendConfig, storeHelper StoreHelper, logger log.Logger) (_ *engineManager, err error) { var duplicateDB *pebble.DB defer func() { @@ -94,12 +96,17 @@ func newEngineManager(config BackendConfig, storeHelper StoreHelper, logger log. alloc.RefCnt = new(atomic.Int64) LastAlloc = alloc } + var opts = make([]membuf.Option, 0, 1) + if !inMemTest { + // otherwise, we use the default allocator that can be tracked by golang runtime. + opts = append(opts, membuf.WithAllocator(alloc)) + } return &engineManager{ BackendConfig: config, StoreHelper: storeHelper, engines: sync.Map{}, externalEngine: map[uuid.UUID]common.Engine{}, - bufferPool: membuf.NewPool(membuf.WithAllocator(alloc)), + bufferPool: membuf.NewPool(opts...), duplicateDB: duplicateDB, keyAdapter: keyAdapter, logger: logger, diff --git a/pkg/lightning/backend/local/local.go b/pkg/lightning/backend/local/local.go index 8675e1bec8a08..b1ee6b8c21862 100644 --- a/pkg/lightning/backend/local/local.go +++ b/pkg/lightning/backend/local/local.go @@ -1632,6 +1632,17 @@ func (local *Backend) SwitchModeByKeyRanges(ctx context.Context, ranges []common } func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, tikvCodec tikvclient.Codec, cacheSize int64, kvBuffer *membuf.Buffer) (*Writer, error) { + // pre-allocate a long enough buffer to avoid a lot of runtime.growslice + // this can help save about 3% of CPU. + var preAllocWriteBatch []common.KvPair + if !cfg.Local.IsKVSorted { + preAllocWriteBatch = make([]common.KvPair, units.MiB) + // we want to keep the cacheSize as the whole limit of this local writer, but the + // main memory usage comes from two member: kvBuffer and writeBatch, so we split + // ~10% to writeBatch for !IsKVSorted, which means we estimate the average length + // of KV pairs are 9 times than the size of common.KvPair (9*72B = 648B). + cacheSize = cacheSize * 9 / 10 + } w := &Writer{ engine: engine, memtableSizeLimit: cacheSize, @@ -1639,12 +1650,7 @@ func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, tikvCodec t isKVSorted: cfg.Local.IsKVSorted, isWriteBatchSorted: true, tikvCodec: tikvCodec, - } - // pre-allocate a long enough buffer to avoid a lot of runtime.growslice - // this can help save about 3% of CPU. - // TODO(lance6716): split the cacheSize to take writeBatch into consideration - if !w.isKVSorted { - w.writeBatch = make([]common.KvPair, units.MiB) + writeBatch: preAllocWriteBatch, } engine.localWriters.Store(w, nil) return w, nil diff --git a/pkg/lightning/backend/local/local_test.go b/pkg/lightning/backend/local/local_test.go index e9b5cd0f2dfba..a14e81495357c 100644 --- a/pkg/lightning/backend/local/local_test.go +++ b/pkg/lightning/backend/local/local_test.go @@ -24,6 +24,7 @@ import ( "math/rand" "path" "path/filepath" + "runtime" "sort" "sync" "sync/atomic" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" sst "github.com/pingcap/kvproto/pkg/import_sstpb" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/restore/split" @@ -56,7 +58,9 @@ import ( "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/engine" "github.com/pingcap/tidb/pkg/util/hack" + "github.com/pingcap/tidb/pkg/util/mathutil" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/http" "google.golang.org/grpc" @@ -2383,3 +2387,121 @@ func TestCheckDiskAvail(t *testing.T) { err = checkDiskAvail(ctx, store) require.NoError(t, err) } + +type mockStoreHelper struct{} + +func (mockStoreHelper) GetTS(context.Context) (physical, logical int64, err error) { + return 12345, 67890, nil +} + +func (mockStoreHelper) GetTiKVCodec() tikv.Codec { + c, _ := tikv.NewCodecV2(tikv.ModeTxn, &keyspacepb.KeyspaceMeta{}) + return c +} + +func TestTotalMemoryConsume(t *testing.T) { + t.Skip("this test is manually run to calibrate the real memory usage with TotalMemoryConsume") + + inMemTest = true + getMemoryInUse := func() int64 { + // wait to make test more stable, maybe releasing memory is slow + runtime.GC() + time.Sleep(time.Second) + runtime.GC() + + s := runtime.MemStats{} + runtime.ReadMemStats(&s) + return int64(s.HeapInuse) + } + memInUseBase := getMemoryInUse() + + ctx := context.Background() + cfg := BackendConfig{ + LocalStoreDir: t.TempDir(), + CheckpointEnabled: true, + DupeDetectEnabled: true, + MemTableSize: 100 * units.MiB, + LocalWriterMemCacheSize: 100 * units.MiB, + } + b, err := NewBackendForTest(ctx, cfg, mockStoreHelper{}) + require.NoError(t, err) + + checkMemoryConsume := func(tag string, expected int64) { + expectedMemConsume := expected + require.EqualValues(t, expectedMemConsume, b.TotalMemoryConsume()) + memInUse := getMemoryInUse() + diff := memInUse - memInUseBase + t.Logf("%s, memInUse %d, memInUseBase %d, diff %d", tag, memInUse, memInUseBase, diff) + require.Less(t, mathutil.Abs(diff-expectedMemConsume), int64(10*units.MiB)) + } + + // 1. test local engine write phase + + engineCfg := &backend.EngineConfig{ + Local: backend.LocalEngineConfig{ + BlockSize: 100 * units.MiB, + }, + } + engineID := uuid.New() + err = b.OpenEngine(ctx, engineCfg, engineID) + require.NoError(t, err) + checkMemoryConsume("after open 1 engine", 0) + + writerCfg := &backend.LocalWriterConfig{} + writerCfg.Local.IsKVSorted = false + unsortedWriter, err := b.LocalWriter(ctx, writerCfg, engineID) + require.NoError(t, err) + writerCfg.Local.IsKVSorted = true + sortedWriter, err := b.LocalWriter(ctx, writerCfg, engineID) + require.NoError(t, err) + // 72 B * 1 Mi from unsortedWriter.writeBatch + checkMemoryConsume("after create engine writers", 72*units.MiB) + + err = unsortedWriter.AppendRows(ctx, []string{"a", "b", "c"}, kv.MakeRowsFromKvPairs([]common.KvPair{ + {Key: []byte("k1"), Val: []byte("v1")}, + {Key: []byte("k3"), Val: []byte("v3")}, + {Key: []byte("k2"), Val: []byte("v2")}, + })) + require.NoError(t, err) + err = sortedWriter.AppendRows(ctx, []string{"a", "b", "c"}, kv.MakeRowsFromKvPairs([]common.KvPair{ + {Key: []byte("k4"), Val: []byte("v4")}, + {Key: []byte("k5"), Val: []byte("v5")}, + {Key: []byte("k6"), Val: []byte("v6")}, + })) + require.NoError(t, err) + // 72 MiB from unsortedWriter.writeBatch, 1 MiB from bufferPool of unsortedWriter + checkMemoryConsume("after write a bit rows", 73*units.MiB) + + _, err = unsortedWriter.Close(ctx) + require.NoError(t, err) + _, err = sortedWriter.Close(ctx) + require.NoError(t, err) + // 1 MiB from bufferPool of unsortedWriter + checkMemoryConsume("after close all writers", 1*units.MiB) + + writerCfg = &backend.LocalWriterConfig{} + writerCfg.Local.IsKVSorted = false + unsortedWriter, err = b.LocalWriter(ctx, writerCfg, engineID) + require.NoError(t, err) + // write about 150 MiB data + val := make([]byte, 35) + for i := 0; i < 1024*1024; i++ { + err = unsortedWriter.AppendRows(ctx, []string{"a", "b", "c"}, kv.MakeRowsFromKvPairs([]common.KvPair{ + {Key: []byte(fmt.Sprintf("key_a_%09d", i)), Val: val}, + {Key: []byte(fmt.Sprintf("key_b_%09d", i)), Val: val}, + {Key: []byte(fmt.Sprintf("key_c_%09d", i)), Val: val}, + })) + require.NoError(t, err) + } + + // 119 MiB from bufferPool, 72 B * 2048910 from unsortedWriter.writeBatch + checkMemoryConsume("after write many rows", 272302064) + + _, err = unsortedWriter.Close(ctx) + require.NoError(t, err) + checkMemoryConsume("after close all writers", 119*units.MiB) + + err = b.CloseEngine(ctx, &backend.EngineConfig{}, engineID) + require.NoError(t, err) + b.CloseEngineMgr() +} diff --git a/pkg/parser/model/model.go b/pkg/parser/model/model.go index 2fe93a94945e5..79633b267dcaf 100644 --- a/pkg/parser/model/model.go +++ b/pkg/parser/model/model.go @@ -383,13 +383,12 @@ func IsIndexPrefixCovered(tbInfo *TableInfo, index *IndexInfo, cols ...CIStr) bo // for use of execution phase. const ExtraHandleID = -1 -// ExtraPidColID is the column ID of column which store the partitionID decoded in global index values. -const ExtraPidColID = -2 +// Deprecated: Use ExtraPhysTblID instead. +// const ExtraPidColID = -2 // ExtraPhysTblID is the column ID of column that should be filled in with the physical table id. // Primarily used for table partition dynamic prune mode, to return which partition (physical table id) the row came from. -// Using a dedicated id for this, since in the future ExtraPidColID and ExtraPhysTblID may be used for the same request. -// Must be after ExtraPidColID! +// If used with a global index, the partition ID decoded from the key value will be filled in. const ExtraPhysTblID = -3 // ExtraRowChecksumID is the column ID of column which holds the row checksum info. @@ -435,8 +434,8 @@ const ( // ExtraHandleName is the name of ExtraHandle Column. var ExtraHandleName = NewCIStr("_tidb_rowid") -// ExtraPartitionIdName is the name of ExtraPartitionId Column. -var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive +// Deprecated: Use ExtraPhysTblIdName instead. +// var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive // ExtraPhysTblIdName is the name of ExtraPhysTblID Column. var ExtraPhysTblIdName = NewCIStr("_tidb_tid") //nolint:revive @@ -923,21 +922,6 @@ func NewExtraHandleColInfo() *ColumnInfo { return colInfo } -// NewExtraPartitionIDColInfo mocks a column info for extra partition id column. -func NewExtraPartitionIDColInfo() *ColumnInfo { - colInfo := &ColumnInfo{ - ID: ExtraPidColID, - Name: ExtraPartitionIdName, - } - colInfo.SetType(mysql.TypeLonglong) - flen, decimal := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong) - colInfo.SetFlen(flen) - colInfo.SetDecimal(decimal) - colInfo.SetCharset(charset.CharsetBin) - colInfo.SetCollate(charset.CollationBin) - return colInfo -} - // NewExtraPhysTblIDColInfo mocks a column info for extra partition id column. func NewExtraPhysTblIDColInfo() *ColumnInfo { colInfo := &ColumnInfo{ diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 84160875c791b..655b27b77cb73 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -168,7 +168,6 @@ go_library( "//pkg/util/mock", "//pkg/util/paging", "//pkg/util/parser", - "//pkg/util/plancache", "//pkg/util/plancodec", "//pkg/util/ranger", "//pkg/util/ranger/context", @@ -292,12 +291,9 @@ go_test( "//pkg/util/context", "//pkg/util/dbterror", "//pkg/util/dbterror/plannererrors", - "//pkg/util/hack", "//pkg/util/hint", - "//pkg/util/kvcache", "//pkg/util/logutil", "//pkg/util/mock", - "//pkg/util/plancache", "//pkg/util/plancodec", "//pkg/util/ranger", "//pkg/util/set", diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 77047dceb3b04..5d4df34ac6c01 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -1423,15 +1423,6 @@ func (ds *DataSource) FindBestTask(prop *property.PhysicalProperty, planCounter } } if canConvertPointGet { - // If the schema contains ExtraPidColID, do not convert to point get. - // Because the point get executor can not handle the extra partition ID column now. - // I.e. Global Index is used - for _, col := range ds.schema.Columns { - if col.ID == model.ExtraPidColID { - canConvertPointGet = false - break - } - } if path != nil && path.Index != nil && path.Index.Global { // Don't convert to point get during ddl // TODO: Revisit truncate partition and global index @@ -2148,6 +2139,15 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub } } + var extraPhysTblCol *expression.Column + // If `dataSouceSchema` contains `model.ExtraPhysTblID`, we should add it into `indexScan.schema` + for _, col := range is.dataSourceSchema.Columns { + if col.ID == model.ExtraPhysTblID { + extraPhysTblCol = col.Clone().(*expression.Column) + break + } + } + if isDoubleRead || is.Index.Global { // If it's double read case, the first index must return handle. So we should add extra handle column // if there isn't a handle column. @@ -2161,23 +2161,19 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub }) } } - // If it's global index, handle and PidColID columns has to be added, so that needed pids can be filtered. - if is.Index.Global { + // If it's global index, handle and PhysTblID columns has to be added, so that needed pids can be filtered. + if is.Index.Global && extraPhysTblCol == nil { indexCols = append(indexCols, &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), - ID: model.ExtraPidColID, + ID: model.ExtraPhysTblID, UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(), - OrigName: model.ExtraPartitionIdName.O, + OrigName: model.ExtraPhysTblIdName.O, }) } } - // If `dataSouceSchema` contains `model.ExtraPhysTblID`, we should add it into `indexScan.schema` - for _, col := range is.dataSourceSchema.Columns { - if col.ID == model.ExtraPhysTblID { - indexCols = append(indexCols, col.Clone().(*expression.Column)) - break - } + if extraPhysTblCol != nil { + indexCols = append(indexCols, extraPhysTblCol) } is.SetSchema(expression.NewSchema(indexCols...)) @@ -2189,14 +2185,14 @@ func (is *PhysicalIndexScan) addSelectionConditionForGlobalIndex(p *DataSource, } args := make([]expression.Expression, 0, len(p.partitionNames)+1) for _, col := range is.schema.Columns { - if col.ID == model.ExtraPidColID { + if col.ID == model.ExtraPhysTblID { args = append(args, col.Clone()) break } } if len(args) != 1 { - return nil, errors.Errorf("Can't find column %s in schema %s", model.ExtraPartitionIdName.O, is.schema) + return nil, errors.Errorf("Can't find column %s in schema %s", model.ExtraPhysTblIdName.O, is.schema) } // For SQL like 'select x from t partition(p0, p1) use index(idx)', diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 8ecac0eef13a2..8f995774c6644 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -3892,7 +3892,7 @@ func unfoldWildStar(field *ast.SelectField, outputName types.NameSlice, column [ } if (dbName.L == "" || dbName.L == name.DBName.L) && (tblName.L == "" || tblName.L == name.TblName.L) && - col.ID != model.ExtraHandleID && col.ID != model.ExtraPidColID && col.ID != model.ExtraPhysTblID { + col.ID != model.ExtraHandleID && col.ID != model.ExtraPhysTblID { colName := &ast.ColumnNameExpr{ Name: &ast.ColumnName{ Schema: name.DBName, diff --git a/pkg/planner/core/plan_cache.go b/pkg/planner/core/plan_cache.go index 1eb35bac878c7..c188698d3c468 100644 --- a/pkg/planner/core/plan_cache.go +++ b/pkg/planner/core/plan_cache.go @@ -37,7 +37,6 @@ import ( contextutil "github.com/pingcap/tidb/pkg/util/context" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" "github.com/pingcap/tidb/pkg/util/kvcache" - utilpc "github.com/pingcap/tidb/pkg/util/plancache" ) // PlanCacheKeyTestIssue43667 is only for test. @@ -174,7 +173,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, return nil, nil, err } - var cacheKey kvcache.Key + var cacheKey string sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx cacheEnabled := false @@ -218,7 +217,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, } } - var matchOpts *utilpc.PlanCacheMatchOpts + var matchOpts *PlanCacheMatchOpts if stmtCtx.UseCache() { var cacheVal kvcache.Value var hit, isPointPlan bool @@ -247,7 +246,7 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, } func adjustCachedPlan(sctx sessionctx.Context, cachedVal *PlanCacheValue, isNonPrepared, isPointPlan bool, - cacheKey kvcache.Key, bindSQL string, is infoschema.InfoSchema, stmt *PlanCacheStmt) (base.Plan, + cacheKey string, bindSQL string, is infoschema.InfoSchema, stmt *PlanCacheStmt) (base.Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx @@ -286,8 +285,8 @@ func adjustCachedPlan(sctx sessionctx.Context, cachedVal *PlanCacheValue, isNonP // generateNewPlan call the optimizer to generate a new plan for current statement // and try to add it to cache func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, - stmt *PlanCacheStmt, cacheKey kvcache.Key, latestSchemaVersion int64, bindSQL string, - matchOpts *utilpc.PlanCacheMatchOpts) (base.Plan, []*types.FieldName, error) { + stmt *PlanCacheStmt, cacheKey string, latestSchemaVersion int64, bindSQL string, + matchOpts *PlanCacheMatchOpts) (base.Plan, []*types.FieldName, error) { stmtAst := stmt.PreparedAst sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx diff --git a/pkg/planner/core/plan_cache_lru.go b/pkg/planner/core/plan_cache_lru.go index 87ef3edc674d5..6f1e8641c30f4 100644 --- a/pkg/planner/core/plan_cache_lru.go +++ b/pkg/planner/core/plan_cache_lru.go @@ -19,17 +19,15 @@ import ( "github.com/pingcap/errors" core_metrics "github.com/pingcap/tidb/pkg/planner/core/metrics" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/kvcache" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/memory" - utilpc "github.com/pingcap/tidb/pkg/util/plancache" "github.com/pingcap/tidb/pkg/util/syncutil" ) // planCacheEntry wraps Key and Value. It's the value of list.Element. type planCacheEntry struct { - PlanKey kvcache.Key + PlanKey string PlanValue kvcache.Value } @@ -39,7 +37,7 @@ func (e *planCacheEntry) MemoryUsage() (sum int64) { return } - return e.PlanKey.(*planCacheKey).MemoryUsage() + e.PlanValue.(*PlanCacheValue).MemoryUsage() + return int64(len(e.PlanKey)) + e.PlanValue.(*PlanCacheValue).MemoryUsage() } // LRUPlanCache is a dedicated least recently used cache, Only used for plan cache. @@ -52,7 +50,7 @@ type LRUPlanCache struct { // lock make cache thread safe lock syncutil.RWMutex // onEvict will be called if any eviction happened, only for test use now - onEvict func(kvcache.Key, kvcache.Value) + onEvict func(string, any) // 0 indicates no quota quota uint64 @@ -80,20 +78,12 @@ func NewLRUPlanCache(capacity uint, guard float64, quota uint64, sctx sessionctx } } -// strHashKey control deep or Shallow copy of string -func strHashKey(key kvcache.Key, deepCopy bool) string { - if deepCopy { - return string(key.Hash()) - } - return string(hack.String(key.Hash())) -} - // Get tries to find the corresponding value according to the given key. -func (l *LRUPlanCache) Get(key kvcache.Key, opts *utilpc.PlanCacheMatchOpts) (value kvcache.Value, ok bool) { +func (l *LRUPlanCache) Get(key string, opts any) (value any, ok bool) { l.lock.RLock() defer l.lock.RUnlock() - bucket, bucketExist := l.buckets[strHashKey(key, false)] + bucket, bucketExist := l.buckets[key] if bucketExist { if element, exist := l.pickFromBucket(bucket, opts); exist { l.lruList.MoveToFront(element) @@ -104,12 +94,11 @@ func (l *LRUPlanCache) Get(key kvcache.Key, opts *utilpc.PlanCacheMatchOpts) (va } // Put puts the (key, value) pair into the LRU Cache. -func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, opts *utilpc.PlanCacheMatchOpts) { +func (l *LRUPlanCache) Put(key string, value, opts any) { l.lock.Lock() defer l.lock.Unlock() - hash := strHashKey(key, true) - bucket, bucketExist := l.buckets[hash] + bucket, bucketExist := l.buckets[key] if bucketExist { if element, exist := l.pickFromBucket(bucket, opts); exist { l.updateInstanceMetric(&planCacheEntry{PlanKey: key, PlanValue: value}, element.Value.(*planCacheEntry)) @@ -118,7 +107,7 @@ func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, opts *utilpc.Pl return } } else { - l.buckets[hash] = make(map[*list.Element]struct{}, 1) + l.buckets[key] = make(map[*list.Element]struct{}, 1) } newCacheEntry := &planCacheEntry{ @@ -126,7 +115,7 @@ func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, opts *utilpc.Pl PlanValue: value, } element := l.lruList.PushFront(newCacheEntry) - l.buckets[hash][element] = struct{}{} + l.buckets[key][element] = struct{}{} l.size++ l.updateInstanceMetric(newCacheEntry, nil) if l.size > l.capacity { @@ -136,19 +125,18 @@ func (l *LRUPlanCache) Put(key kvcache.Key, value kvcache.Value, opts *utilpc.Pl } // Delete deletes the multi-values from the LRU Cache. -func (l *LRUPlanCache) Delete(key kvcache.Key) { +func (l *LRUPlanCache) Delete(key string) { l.lock.Lock() defer l.lock.Unlock() - hash := strHashKey(key, false) - bucket, bucketExist := l.buckets[hash] + bucket, bucketExist := l.buckets[key] if bucketExist { for element := range bucket { l.updateInstanceMetric(nil, element.Value.(*planCacheEntry)) l.lruList.Remove(element) l.size-- } - delete(l.buckets, hash) + delete(l.buckets, key) } } @@ -229,7 +217,7 @@ func (l *LRUPlanCache) removeOldest() { // removeFromBucket remove element from bucket func (l *LRUPlanCache) removeFromBucket(element *list.Element) { - hash := strHashKey(element.Value.(*planCacheEntry).PlanKey, false) + hash := element.Value.(*planCacheEntry).PlanKey bucket := l.buckets[hash] delete(bucket, element) if len(bucket) == 0 { @@ -251,7 +239,11 @@ func (l *LRUPlanCache) memoryControl() { } // PickPlanFromBucket pick one plan from bucket -func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, matchOpts *utilpc.PlanCacheMatchOpts) (*list.Element, bool) { +func (l *LRUPlanCache) pickFromBucket(bucket map[*list.Element]struct{}, opts any) (*list.Element, bool) { + var matchOpts *PlanCacheMatchOpts + if opts != nil { + matchOpts = opts.(*PlanCacheMatchOpts) + } for k := range bucket { if matchCachedPlan(l.sctx, k.Value.(*planCacheEntry).PlanValue.(*PlanCacheValue), matchOpts) { return k, true diff --git a/pkg/planner/core/plan_cache_lru_test.go b/pkg/planner/core/plan_cache_lru_test.go index d737fa55a4ad1..a522c42b95f14 100644 --- a/pkg/planner/core/plan_cache_lru_test.go +++ b/pkg/planner/core/plan_cache_lru_test.go @@ -14,8 +14,8 @@ package core import ( + "fmt" "math/rand" - "strconv" "testing" "time" @@ -23,18 +23,12 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/hack" - "github.com/pingcap/tidb/pkg/util/kvcache" - utilpc "github.com/pingcap/tidb/pkg/util/plancache" "github.com/stretchr/testify/require" ) -func randomPlanCacheKey() *planCacheKey { +func randomPlanCacheKey() string { random := rand.New(rand.NewSource(time.Now().UnixNano())) - return &planCacheKey{ - database: strconv.FormatInt(int64(random.Int()), 10), - schemaVersion: time.Now().UnixNano(), - } + return fmt.Sprintf("%v", random.Int()) } func randomPlanCacheValue(types []*types.FieldType) *PlanCacheValue { @@ -44,7 +38,7 @@ func randomPlanCacheValue(types []*types.FieldType) *PlanCacheValue { random := rand.New(rand.NewSource(time.Now().UnixNano())) return &PlanCacheValue{ Plan: plans[random.Int()%len(plans)], - matchOpts: &utilpc.PlanCacheMatchOpts{ParamTypes: types}, + matchOpts: &PlanCacheMatchOpts{ParamTypes: types}, } } @@ -58,14 +52,14 @@ func TestLRUPCPut(t *testing.T) { lruA := NewLRUPlanCache(0, 0, 0, mockCtx, false) require.Equal(t, lruA.capacity, uint(100)) - maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value) + dropCnt := 0 lru := NewLRUPlanCache(3, 0, 0, mockCtx, false) - lru.onEvict = func(key kvcache.Key, value kvcache.Value) { - maxMemDroppedKv[key] = value + lru.onEvict = func(key string, value any) { + dropCnt++ } require.Equal(t, uint(3), lru.capacity) - keys := make([]*planCacheKey, 5) + keys := make([]string, 5) vals := make([]*PlanCacheValue, 5) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, @@ -79,8 +73,8 @@ func TestLRUPCPut(t *testing.T) { // one key corresponding to multi values for i := 0; i < 5; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(1), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = "key-1" + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -93,14 +87,13 @@ func TestLRUPCPut(t *testing.T) { require.Equal(t, uint(3), lru.size) // test for non-existent elements - require.Len(t, maxMemDroppedKv, 2) + require.Equal(t, dropCnt, 2) for i := 0; i < 2; i++ { - bucket, exist := lru.buckets[string(hack.String(keys[i].Hash()))] + bucket, exist := lru.buckets[keys[i]] require.True(t, exist) for element := range bucket { require.NotEqual(t, vals[i], element.Value.(*planCacheEntry).PlanValue) } - require.Equal(t, vals[i], maxMemDroppedKv[keys[i]]) } // test for existent elements @@ -116,9 +109,9 @@ func TestLRUPCPut(t *testing.T) { require.NotNil(t, key) require.Equal(t, keys[i], key) - bucket, exist := lru.buckets[string(hack.String(keys[i].Hash()))] + bucket, exist := lru.buckets[keys[i]] require.True(t, exist) - matchOpts := &utilpc.PlanCacheMatchOpts{ + matchOpts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -147,7 +140,7 @@ func TestLRUPCGet(t *testing.T) { }() lru := NewLRUPlanCache(3, 0, 0, mockCtx, false) - keys := make([]*planCacheKey, 5) + keys := make([]string, 5) vals := make([]*PlanCacheValue, 5) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, @@ -160,8 +153,8 @@ func TestLRUPCGet(t *testing.T) { } // 5 bucket for i := 0; i < 5; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i%4), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = fmt.Sprintf("key-%v", i%4) + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -173,7 +166,7 @@ func TestLRUPCGet(t *testing.T) { // test for non-existent elements for i := 0; i < 2; i++ { - opts := &utilpc.PlanCacheMatchOpts{ + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -183,7 +176,7 @@ func TestLRUPCGet(t *testing.T) { } for i := 2; i < 5; i++ { - opts := &utilpc.PlanCacheMatchOpts{ + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -215,7 +208,7 @@ func TestLRUPCDelete(t *testing.T) { }() lru := NewLRUPlanCache(3, 0, 0, mockCtx, false) - keys := make([]*planCacheKey, 3) + keys := make([]string, 3) vals := make([]*PlanCacheValue, 3) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, @@ -225,8 +218,8 @@ func TestLRUPCDelete(t *testing.T) { {1}, {2}, {3}, } for i := 0; i < 3; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = fmt.Sprintf("key-%v", i) + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: limitParams[i], } @@ -239,7 +232,7 @@ func TestLRUPCDelete(t *testing.T) { lru.Delete(keys[1]) - value, exists := lru.Get(keys[1], &utilpc.PlanCacheMatchOpts{ + value, exists := lru.Get(keys[1], &PlanCacheMatchOpts{ ParamTypes: pTypes[1], LimitOffsetAndCount: limitParams[1], }) @@ -247,13 +240,13 @@ func TestLRUPCDelete(t *testing.T) { require.Nil(t, value) require.Equal(t, 2, int(lru.size)) - _, exists = lru.Get(keys[0], &utilpc.PlanCacheMatchOpts{ + _, exists = lru.Get(keys[0], &PlanCacheMatchOpts{ ParamTypes: pTypes[0], LimitOffsetAndCount: limitParams[0], }) require.True(t, exists) - _, exists = lru.Get(keys[2], &utilpc.PlanCacheMatchOpts{ + _, exists = lru.Get(keys[2], &PlanCacheMatchOpts{ ParamTypes: pTypes[2], LimitOffsetAndCount: limitParams[2], }) @@ -266,15 +259,15 @@ func TestLRUPCDeleteAll(t *testing.T) { defer func() { domain.GetDomain(ctx).StatsHandle().Close() }() - keys := make([]*planCacheKey, 3) + keys := make([]string, 3) vals := make([]*PlanCacheValue, 3) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDate)}, } for i := 0; i < 3; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = fmt.Sprintf("key-%v", i) + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: []uint64{}, } @@ -288,7 +281,7 @@ func TestLRUPCDeleteAll(t *testing.T) { lru.DeleteAll() for i := 0; i < 3; i++ { - opts := &utilpc.PlanCacheMatchOpts{ + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: []uint64{}, } @@ -300,18 +293,18 @@ func TestLRUPCDeleteAll(t *testing.T) { } func TestLRUPCSetCapacity(t *testing.T) { - maxMemDroppedKv := make(map[kvcache.Key]kvcache.Value) ctx := MockContext() lru := NewLRUPlanCache(5, 0, 0, ctx, false) defer func() { domain.GetDomain(ctx).StatsHandle().Close() }() - lru.onEvict = func(key kvcache.Key, value kvcache.Value) { - maxMemDroppedKv[key] = value + dropCnt := 0 + lru.onEvict = func(key string, value any) { + dropCnt++ } require.Equal(t, uint(5), lru.capacity) - keys := make([]*planCacheKey, 5) + keys := make([]string, 5) vals := make([]*PlanCacheValue, 5) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, @@ -322,8 +315,8 @@ func TestLRUPCSetCapacity(t *testing.T) { // one key corresponding to multi values for i := 0; i < 5; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(1), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = "key-1" + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: []uint64{}, } @@ -339,14 +332,13 @@ func TestLRUPCSetCapacity(t *testing.T) { require.NoError(t, err) // test for non-existent elements - require.Len(t, maxMemDroppedKv, 2) + require.Equal(t, dropCnt, 2) for i := 0; i < 2; i++ { - bucket, exist := lru.buckets[string(hack.String(keys[i].Hash()))] + bucket, exist := lru.buckets[keys[i]] require.True(t, exist) for element := range bucket { require.NotEqual(t, vals[i], element.Value.(*planCacheEntry).PlanValue) } - require.Equal(t, vals[i], maxMemDroppedKv[keys[i]]) } // test for existent elements @@ -379,8 +371,8 @@ func TestIssue37914(t *testing.T) { domain.GetDomain(ctx).StatsHandle().Close() }() pTypes := []*types.FieldType{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)} - key := &planCacheKey{database: strconv.FormatInt(int64(1), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + key := "key-1" + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes, LimitOffsetAndCount: []uint64{}, } @@ -399,7 +391,7 @@ func TestIssue38244(t *testing.T) { }() require.Equal(t, uint(3), lru.capacity) - keys := make([]*planCacheKey, 5) + keys := make([]string, 5) vals := make([]*PlanCacheValue, 5) pTypes := [][]*types.FieldType{{types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeDouble)}, {types.NewFieldType(mysql.TypeFloat), types.NewFieldType(mysql.TypeEnum)}, @@ -410,8 +402,8 @@ func TestIssue38244(t *testing.T) { // one key corresponding to multi values for i := 0; i < 5; i++ { - keys[i] = &planCacheKey{database: strconv.FormatInt(int64(i), 10)} - opts := &utilpc.PlanCacheMatchOpts{ + keys[i] = fmt.Sprintf("key-%v", i) + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes[i], LimitOffsetAndCount: []uint64{}, } @@ -431,8 +423,8 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) { }() ctx.GetSessionVars().EnablePreparedPlanCacheMemoryMonitor = true lru := NewLRUPlanCache(3, 0, 0, ctx, false) - evict := make(map[kvcache.Key]kvcache.Value) - lru.onEvict = func(key kvcache.Key, value kvcache.Value) { + evict := make(map[string]any) + lru.onEvict = func(key string, value any) { evict[key] = value } var res int64 = 0 @@ -440,31 +432,31 @@ func TestLRUPlanCacheMemoryUsage(t *testing.T) { for i := 0; i < 3; i++ { k := randomPlanCacheKey() v := randomPlanCacheValue(pTypes) - opts := &utilpc.PlanCacheMatchOpts{ + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes, LimitOffsetAndCount: []uint64{}, } lru.Put(k, v, opts) - res += k.MemoryUsage() + v.MemoryUsage() + res += int64(len(k)) + v.MemoryUsage() require.Equal(t, lru.MemoryUsage(), res) } // evict p := &PhysicalTableScan{} - k := &planCacheKey{database: "3"} + k := "key-3" v := &PlanCacheValue{Plan: p} - opts := &utilpc.PlanCacheMatchOpts{ + opts := &PlanCacheMatchOpts{ ParamTypes: pTypes, LimitOffsetAndCount: []uint64{}, } lru.Put(k, v, opts) - res += k.MemoryUsage() + v.MemoryUsage() + res += int64(len(k)) + v.MemoryUsage() for kk, vv := range evict { - res -= kk.(*planCacheKey).MemoryUsage() + vv.(*PlanCacheValue).MemoryUsage() + res -= int64(len(kk)) + vv.(*PlanCacheValue).MemoryUsage() } require.Equal(t, lru.MemoryUsage(), res) // delete lru.Delete(k) - res -= k.MemoryUsage() + v.MemoryUsage() + res -= int64(len(k)) + v.MemoryUsage() require.Equal(t, lru.MemoryUsage(), res) // delete all lru.DeleteAll() diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index 39d41242b6bc6..1e56a31c1fb30 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -22,7 +22,6 @@ import ( "sort" "strconv" "time" - "unsafe" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/bindinfo" @@ -46,9 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/hint" "github.com/pingcap/tidb/pkg/util/intest" - "github.com/pingcap/tidb/pkg/util/kvcache" "github.com/pingcap/tidb/pkg/util/logutil" - utilpc "github.com/pingcap/tidb/pkg/util/plancache" "github.com/pingcap/tidb/pkg/util/size" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -229,38 +226,6 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, return preparedObj, p, paramCount, nil } -// planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text. -// Put the parameters that may affect the plan in planCacheValue. -// However, due to some compatibility reasons, we will temporarily keep some system variable-related values in planCacheKey. -// At the same time, because these variables have a small impact on plan, we will move them to PlanCacheValue later if necessary. -// TODO: maintain a sync.pool for this structure. -type planCacheKey struct { - database string - connID uint64 - stmtText string - schemaVersion int64 - tblVersionMap map[int64]uint64 - - // Only be set in rc or for update read and leave it default otherwise. - // In Rc or ForUpdateRead, we should check whether the information schema has been changed when using plan cache. - // If it changed, we should rebuild the plan. lastUpdatedSchemaVersion help us to decide whether we should rebuild - // the plan in rc or for update read. - lastUpdatedSchemaVersion int64 - sqlMode mysql.SQLMode - timezoneOffset int - isolationReadEngines map[kv.StoreType]struct{} - selectLimit uint64 - bindSQL string - connCollation string - inRestrictedSQL bool - restrictedReadOnly bool - TiDBSuperReadOnly bool - exprBlacklistTS int64 // expr-pushdown-blacklist can affect query optimization, so we need to consider it in plan cache. - - memoryUsage int64 // Do not include in hash - hash []byte -} - func hashInt64Uint64Map(b []byte, m map[int64]uint64) []byte { keys := make([]int64, 0, len(m)) for k := range m { @@ -278,82 +243,16 @@ func hashInt64Uint64Map(b []byte, m map[int64]uint64) []byte { return b } -// Hash implements Key interface. -func (key *planCacheKey) Hash() []byte { - if len(key.hash) == 0 { - if key.hash == nil { - key.hash = make([]byte, 0, len(key.stmtText)*2) - } - key.hash = append(key.hash, hack.Slice(key.database)...) - key.hash = codec.EncodeInt(key.hash, int64(key.connID)) - key.hash = append(key.hash, hack.Slice(key.stmtText)...) - key.hash = codec.EncodeInt(key.hash, key.schemaVersion) - key.hash = hashInt64Uint64Map(key.hash, key.tblVersionMap) - key.hash = codec.EncodeInt(key.hash, key.lastUpdatedSchemaVersion) - key.hash = codec.EncodeInt(key.hash, int64(key.sqlMode)) - key.hash = codec.EncodeInt(key.hash, int64(key.timezoneOffset)) - if _, ok := key.isolationReadEngines[kv.TiDB]; ok { - key.hash = append(key.hash, kv.TiDB.Name()...) - } - if _, ok := key.isolationReadEngines[kv.TiKV]; ok { - key.hash = append(key.hash, kv.TiKV.Name()...) - } - if _, ok := key.isolationReadEngines[kv.TiFlash]; ok { - key.hash = append(key.hash, kv.TiFlash.Name()...) - } - key.hash = codec.EncodeInt(key.hash, int64(key.selectLimit)) - key.hash = append(key.hash, hack.Slice(key.bindSQL)...) - key.hash = append(key.hash, hack.Slice(key.connCollation)...) - key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.inRestrictedSQL))...) - key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.restrictedReadOnly))...) - key.hash = append(key.hash, hack.Slice(strconv.FormatBool(key.TiDBSuperReadOnly))...) - key.hash = codec.EncodeInt(key.hash, key.exprBlacklistTS) - } - return key.hash -} - -const emptyPlanCacheKeySize = int64(unsafe.Sizeof(planCacheKey{})) - -// MemoryUsage return the memory usage of planCacheKey -func (key *planCacheKey) MemoryUsage() (sum int64) { - if key == nil { - return - } - - if key.memoryUsage > 0 { - return key.memoryUsage - } - sum = emptyPlanCacheKeySize + int64(len(key.database)+len(key.stmtText)+len(key.bindSQL)+len(key.connCollation)) + - int64(len(key.isolationReadEngines))*size.SizeOfUint8 + int64(cap(key.hash)) - key.memoryUsage = sum - return -} - -// SetPstmtIDSchemaVersion implements PstmtCacheKeyMutator interface to change pstmtID and schemaVersion of cacheKey. -// so we can reuse Key instead of new every time. -func SetPstmtIDSchemaVersion(key kvcache.Key, stmtText string, schemaVersion int64, isolationReadEngines map[kv.StoreType]struct{}) { - psStmtKey, isPsStmtKey := key.(*planCacheKey) - if !isPsStmtKey { - return - } - psStmtKey.stmtText = stmtText - psStmtKey.schemaVersion = schemaVersion - psStmtKey.isolationReadEngines = make(map[kv.StoreType]struct{}) - for k, v := range isolationReadEngines { - psStmtKey.isolationReadEngines[k] = v - } - psStmtKey.hash = psStmtKey.hash[:0] -} - -// NewPlanCacheKey creates a new planCacheKey object. +// NewPlanCacheKey creates the plan cache key for this statement. // Note: lastUpdatedSchemaVersion will only be set in the case of rc or for update read in order to // differentiate the cache key. In other cases, it will be 0. -func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, schemaVersion, lastUpdatedSchemaVersion int64, bindSQL string, exprBlacklistTS int64, relatedSchemaVersion map[int64]uint64) (kvcache.Key, error) { +// All information that might affect the plan should be considered in this function. +func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, schemaVersion, lastUpdatedSchemaVersion int64, bindSQL string, exprBlacklistTS int64, relatedSchemaVersion map[int64]uint64) (string, error) { if stmtText == "" { - return nil, errors.New("no statement text") + return "", errors.New("no statement text") } if schemaVersion == 0 && !intest.InTest { - return nil, errors.New("Schema version uninitialized") + return "", errors.New("Schema version uninitialized") } if stmtDB == "" { stmtDB = sessionVars.CurrentDB @@ -364,31 +263,37 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, stmtText, stmtDB string, } _, connCollation := sessionVars.GetCharsetInfo() - key := &planCacheKey{ - database: stmtDB, - connID: sessionVars.ConnectionID, - stmtText: stmtText, - schemaVersion: schemaVersion, - tblVersionMap: make(map[int64]uint64), - lastUpdatedSchemaVersion: lastUpdatedSchemaVersion, - sqlMode: sessionVars.SQLMode, - timezoneOffset: timezoneOffset, - isolationReadEngines: make(map[kv.StoreType]struct{}), - selectLimit: sessionVars.SelectLimit, - bindSQL: bindSQL, - connCollation: connCollation, - inRestrictedSQL: sessionVars.InRestrictedSQL, - restrictedReadOnly: variable.RestrictedReadOnly.Load(), - TiDBSuperReadOnly: variable.VarTiDBSuperReadOnly.Load(), - exprBlacklistTS: exprBlacklistTS, - } - for k, v := range sessionVars.IsolationReadEngines { - key.isolationReadEngines[k] = v - } - for k, v := range relatedSchemaVersion { - key.tblVersionMap[k] = v - } - return key, nil + hash := make([]byte, 0, len(stmtText)*2) + hash = append(hash, hack.Slice(stmtDB)...) + hash = codec.EncodeInt(hash, int64(sessionVars.ConnectionID)) + hash = append(hash, hack.Slice(stmtText)...) + hash = codec.EncodeInt(hash, schemaVersion) + hash = hashInt64Uint64Map(hash, relatedSchemaVersion) + // Only be set in rc or for update read and leave it default otherwise. + // In Rc or ForUpdateRead, we should check whether the information schema has been changed when using plan cache. + // If it changed, we should rebuild the plan. lastUpdatedSchemaVersion help us to decide whether we should rebuild + // the plan in rc or for update read. + hash = codec.EncodeInt(hash, lastUpdatedSchemaVersion) + hash = codec.EncodeInt(hash, int64(sessionVars.SQLMode)) + hash = codec.EncodeInt(hash, int64(timezoneOffset)) + if _, ok := sessionVars.IsolationReadEngines[kv.TiDB]; ok { + hash = append(hash, kv.TiDB.Name()...) + } + if _, ok := sessionVars.IsolationReadEngines[kv.TiKV]; ok { + hash = append(hash, kv.TiKV.Name()...) + } + if _, ok := sessionVars.IsolationReadEngines[kv.TiFlash]; ok { + hash = append(hash, kv.TiFlash.Name()...) + } + hash = codec.EncodeInt(hash, int64(sessionVars.SelectLimit)) + hash = append(hash, hack.Slice(bindSQL)...) + hash = append(hash, hack.Slice(connCollation)...) + hash = append(hash, hack.Slice(strconv.FormatBool(sessionVars.InRestrictedSQL))...) + hash = append(hash, hack.Slice(strconv.FormatBool(variable.RestrictedReadOnly.Load()))...) + hash = append(hash, hack.Slice(strconv.FormatBool(variable.VarTiDBSuperReadOnly.Load()))...) + // expr-pushdown-blacklist can affect query optimization, so we need to consider it in plan cache. + hash = codec.EncodeInt(hash, exprBlacklistTS) + return string(hash), nil } // PlanCacheValue stores the cached Statement and StmtNode. @@ -399,7 +304,7 @@ type PlanCacheValue struct { memoryUsage int64 // matchOpts stores some fields help to choose a suitable plan - matchOpts *utilpc.PlanCacheMatchOpts + matchOpts *PlanCacheMatchOpts // stmtHints stores the hints which set session variables, because the hints won't be processed using cached plan. stmtHints *hint.StmtHints } @@ -448,7 +353,7 @@ func (v *PlanCacheValue) MemoryUsage() (sum int64) { // NewPlanCacheValue creates a SQLCacheValue. func NewPlanCacheValue(plan base.Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, - matchOpts *utilpc.PlanCacheMatchOpts, stmtHints *hint.StmtHints) *PlanCacheValue { + matchOpts *PlanCacheMatchOpts, stmtHints *hint.StmtHints) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { dstMap[k] = v @@ -466,6 +371,23 @@ func NewPlanCacheValue(plan base.Plan, names []*types.FieldName, srcMap map[*mod } } +// PlanCacheMatchOpts store some property used to fetch plan from plan cache +// The structure set here is to avoid import cycle +type PlanCacheMatchOpts struct { + // paramTypes stores all parameters' FieldType, some different parameters may share same plan + ParamTypes []*types.FieldType + // limitOffsetAndCount stores all the offset and key parameters extract from limit statement + // only used for cache and pick plan with parameters in limit + LimitOffsetAndCount []uint64 + // HasSubQuery indicate whether this query has sub query + HasSubQuery bool + // StatsVersionHash is the hash value of the statistics version + StatsVersionHash uint64 + + // Below are some variables that can affect the plan + ForeignKeyChecks bool +} + // PlanCacheQueryFeatures records all query features which may affect plan selection. type PlanCacheQueryFeatures struct { limits []*ast.Limit @@ -566,7 +488,7 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa // GetMatchOpts get options to fetch plan or generate new plan // we can add more options here -func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) *utilpc.PlanCacheMatchOpts { +func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) *PlanCacheMatchOpts { var statsVerHash uint64 var limitOffsetAndCount []uint64 @@ -607,7 +529,7 @@ func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanC } } - return &utilpc.PlanCacheMatchOpts{ + return &PlanCacheMatchOpts{ LimitOffsetAndCount: limitOffsetAndCount, HasSubQuery: stmt.QueryFeatures.hasSubquery, StatsVersionHash: statsVerHash, @@ -752,7 +674,7 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p } // matchCachedPlan checks whether this plan is matched with these match-options. -func matchCachedPlan(sctx sessionctx.Context, value *PlanCacheValue, matchOpts *utilpc.PlanCacheMatchOpts) bool { +func matchCachedPlan(sctx sessionctx.Context, value *PlanCacheValue, matchOpts *PlanCacheMatchOpts) bool { if matchOpts == nil { // if PointGet, the matchOpts is nil return true } diff --git a/pkg/planner/core/plan_to_pb.go b/pkg/planner/core/plan_to_pb.go index de96eb8cb9855..970f13058e003 100644 --- a/pkg/planner/core/plan_to_pb.go +++ b/pkg/planner/core/plan_to_pb.go @@ -465,8 +465,6 @@ func (p *PhysicalIndexScan) ToPB(_ *base.BuildPBContext, _ kv.StoreType) (*tipb. columns = append(columns, model.NewExtraHandleColInfo()) } else if col.ID == model.ExtraPhysTblID { columns = append(columns, model.NewExtraPhysTblIDColInfo()) - } else if col.ID == model.ExtraPidColID { - columns = append(columns, model.NewExtraPartitionIDColInfo()) } else { columns = append(columns, FindColumnInfoByID(tableColumns, col.ID)) } diff --git a/pkg/planner/core/rule_column_pruning.go b/pkg/planner/core/rule_column_pruning.go index 7a920f83c3e3b..4bcae1f906b83 100644 --- a/pkg/planner/core/rule_column_pruning.go +++ b/pkg/planner/core/rule_column_pruning.go @@ -356,7 +356,7 @@ func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt parentUsedCols = append(parentUsedCols, p.handleCols.GetCol(i)) } for _, col := range p.Schema().Columns { - if col.ID == model.ExtraPidColID || col.ID == model.ExtraPhysTblID { + if col.ID == model.ExtraPhysTblID { parentUsedCols = append(parentUsedCols, col) } } diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index cdd2258a77ef1..f6e9e81658ece 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -471,15 +471,6 @@ func (*partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types.Fi }) continue } - if colExpr.ID == model.ExtraPidColID { - names = append(names, &types.FieldName{ - DBName: ds.DBName, - TblName: ds.tableInfo.Name, - ColName: model.ExtraPartitionIdName, - OrigColName: model.ExtraPartitionIdName, - }) - continue - } if colExpr.ID == model.ExtraPhysTblID { names = append(names, &types.FieldName{ DBName: ds.DBName, diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 0742bc53dce39..21c67e23a43c7 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -100,7 +100,6 @@ go_library( "//pkg/util/dbterror/plannererrors", "//pkg/util/execdetails", "//pkg/util/intest", - "//pkg/util/kvcache", "//pkg/util/logutil", "//pkg/util/logutil/consistency", "//pkg/util/memory", diff --git a/pkg/session/session.go b/pkg/session/session.go index ccfc9fb278017..e3bbc1ae000ad 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -106,7 +106,6 @@ import ( "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" "github.com/pingcap/tidb/pkg/util/execdetails" "github.com/pingcap/tidb/pkg/util/intest" - "github.com/pingcap/tidb/pkg/util/kvcache" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/logutil/consistency" "github.com/pingcap/tidb/pkg/util/memory" @@ -293,7 +292,7 @@ func (s *session) cleanRetryInfo() { } planCacheEnabled := s.GetSessionVars().EnablePreparedPlanCache - var cacheKey kvcache.Key + var cacheKey, bindSQL string var err error var preparedObj *plannercore.PlanCacheStmt var stmtText, stmtDB string @@ -303,7 +302,7 @@ func (s *session) cleanRetryInfo() { preparedObj, ok = preparedPointer.(*plannercore.PlanCacheStmt) if ok { stmtText, stmtDB = preparedObj.StmtText, preparedObj.StmtDB - bindSQL, _ := bindinfo.MatchSQLBindingForPlanCache(s.pctx, preparedObj.PreparedAst.Stmt, &preparedObj.BindingInfo) + bindSQL, _ = bindinfo.MatchSQLBindingForPlanCache(s.pctx, preparedObj.PreparedAst.Stmt, &preparedObj.BindingInfo) cacheKey, err = plannercore.NewPlanCacheKey(s.sessionVars, stmtText, stmtDB, preparedObj.SchemaVersion, 0, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load(), preparedObj.RelateVersion) if err != nil { @@ -316,7 +315,12 @@ func (s *session) cleanRetryInfo() { for i, stmtID := range retryInfo.DroppedPreparedStmtIDs { if planCacheEnabled { if i > 0 && preparedObj != nil { - plannercore.SetPstmtIDSchemaVersion(cacheKey, stmtText, preparedObj.SchemaVersion, s.sessionVars.IsolationReadEngines) + cacheKey, err = plannercore.NewPlanCacheKey(s.sessionVars, stmtText, stmtDB, preparedObj.SchemaVersion, + 0, bindSQL, expression.ExprPushDownBlackListReloadTimeStamp.Load(), preparedObj.RelateVersion) + if err != nil { + logutil.Logger(s.currentCtx).Warn("clean cached plan failed", zap.Error(err)) + return + } } if !s.sessionVars.IgnorePreparedCacheCloseStmt { // keep the plan in cache s.GetSessionPlanCache().Delete(cacheKey) @@ -4165,6 +4169,9 @@ func (s *session) GetStmtStats() *stmtstats.StatementStats { // SetMemoryFootprintChangeHook sets the hook that is called when the memdb changes its size. // Call this after s.txn becomes valid, since TxnInfo is initialized when the txn becomes valid. func (s *session) SetMemoryFootprintChangeHook() { + if s.txn.MemHookSet() { + return + } if config.GetGlobalConfig().Performance.TxnTotalSizeLimit != config.DefTxnTotalSizeLimit { // if the user manually specifies the config, don't involve the new memory tracker mechanism, let the old config // work as before. diff --git a/pkg/session/test/session_test.go b/pkg/session/test/session_test.go index db5134cc9fcce..39a0f8a9057d4 100644 --- a/pkg/session/test/session_test.go +++ b/pkg/session/test/session_test.go @@ -410,11 +410,11 @@ func TestStmtHints(t *testing.T) { val = int64(1) * 1024 * 1024 require.True(t, tk.Session().GetSessionVars().MemTracker.CheckBytesLimit(val)) - tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 select /*+ MEMORY_QUOTA(3 MB) */ * from t1;") + tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 select /*+ MEMORY_QUOTA(1 MB) */ * from t1;") val = int64(1) * 1024 * 1024 require.True(t, tk.Session().GetSessionVars().MemTracker.CheckBytesLimit(val)) - require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) - require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3126]Hint MEMORY_QUOTA(`3145728`) is ignored as conflicting/duplicated.") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 2) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[planner:3126]Hint MEMORY_QUOTA(`1048576`) is ignored as conflicting/duplicated.") // Test NO_INDEX_MERGE hint tk.Session().GetSessionVars().SetEnableIndexMerge(true) diff --git a/pkg/session/txn.go b/pkg/session/txn.go index 7f44220780fc6..0ec88c4a5667a 100644 --- a/pkg/session/txn.go +++ b/pkg/session/txn.go @@ -214,6 +214,14 @@ func (txn *LazyTxn) SetMemoryFootprintChangeHook(hook func(uint64)) { txn.Transaction.SetMemoryFootprintChangeHook(hook) } +// MemHookSet returns whether the memory footprint change hook is set. +func (txn *LazyTxn) MemHookSet() bool { + if txn.Transaction == nil { + return false + } + return txn.Transaction.MemHookSet() +} + // Valid implements the kv.Transaction interface. func (txn *LazyTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() diff --git a/pkg/sessionctx/BUILD.bazel b/pkg/sessionctx/BUILD.bazel index 85700c2ff8abb..5a956ad4e974c 100644 --- a/pkg/sessionctx/BUILD.bazel +++ b/pkg/sessionctx/BUILD.bazel @@ -22,8 +22,6 @@ go_library( "//pkg/table/context", "//pkg/util", "//pkg/util/context", - "//pkg/util/kvcache", - "//pkg/util/plancache", "//pkg/util/ranger/context", "//pkg/util/sli", "//pkg/util/sqlexec", diff --git a/pkg/sessionctx/context.go b/pkg/sessionctx/context.go index ad835a4eebff5..6fe2799a37c22 100644 --- a/pkg/sessionctx/context.go +++ b/pkg/sessionctx/context.go @@ -35,8 +35,6 @@ import ( tbctx "github.com/pingcap/tidb/pkg/table/context" "github.com/pingcap/tidb/pkg/util" contextutil "github.com/pingcap/tidb/pkg/util/context" - "github.com/pingcap/tidb/pkg/util/kvcache" - utilpc "github.com/pingcap/tidb/pkg/util/plancache" rangerctx "github.com/pingcap/tidb/pkg/util/ranger/context" "github.com/pingcap/tidb/pkg/util/sli" "github.com/pingcap/tidb/pkg/util/sqlexec" @@ -55,9 +53,9 @@ type SessionStatesHandler interface { // PlanCache is an interface for prepare and non-prepared plan cache type PlanCache interface { - Get(key kvcache.Key, opts *utilpc.PlanCacheMatchOpts) (value kvcache.Value, ok bool) - Put(key kvcache.Key, value kvcache.Value, opts *utilpc.PlanCacheMatchOpts) - Delete(key kvcache.Key) + Get(key string, opts any) (value any, ok bool) + Put(key string, value, opts any) + Delete(key string) DeleteAll() Size() int SetCapacity(capacity uint) error diff --git a/pkg/store/mockstore/unistore/cophandler/closure_exec.go b/pkg/store/mockstore/unistore/cophandler/closure_exec.go index 0a19605d34a99..7ab29cfa7be15 100644 --- a/pkg/store/mockstore/unistore/cophandler/closure_exec.go +++ b/pkg/store/mockstore/unistore/cophandler/closure_exec.go @@ -305,13 +305,6 @@ func (e *closureExecutor) initIdxScanCtx(idxScan *tipb.IndexScan) { lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1] } - // Here it is required that ExtraPidColID - // is after all other columns except ExtraPhysTblID - if lastColumn.GetColumnId() == model.ExtraPidColID { - e.idxScanCtx.columnLen-- - lastColumn = e.columnInfos[e.idxScanCtx.columnLen-1] - } - if len(e.idxScanCtx.primaryColumnIds) == 0 { if lastColumn.GetPkHandle() { if mysql.HasUnsignedFlag(uint(lastColumn.GetFlag())) { @@ -932,7 +925,9 @@ func (e *closureExecutor) indexScanProcessCore(key, value []byte) error { } // Add ExtraPhysTblID if requested // Assumes it is always last! - if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID { + // If we need pid, it already filled by above loop. Because `DecodeIndexKV` func will return pid in `values`. + // The following if statement is to fill in the tid when we needed it. + if e.columnInfos[len(e.columnInfos)-1].ColumnId == model.ExtraPhysTblID && len(e.columnInfos) >= len(values) { tblID := tablecodec.DecodeTableID(key) chk.AppendInt64(len(e.columnInfos)-1, tblID) } diff --git a/pkg/store/mockstore/unistore/cophandler/cop_handler.go b/pkg/store/mockstore/unistore/cophandler/cop_handler.go index 62448f0f0853b..7097f07104476 100644 --- a/pkg/store/mockstore/unistore/cophandler/cop_handler.go +++ b/pkg/store/mockstore/unistore/cophandler/cop_handler.go @@ -19,6 +19,7 @@ import ( "context" "fmt" "strings" + "sync" "time" "github.com/golang/protobuf/proto" @@ -50,6 +51,32 @@ import ( "github.com/pingcap/tipb/go-tipb" ) +var globalLocationMap *locationMap = newLocationMap() + +type locationMap struct { + lmap map[string]*time.Location + mu sync.RWMutex +} + +func newLocationMap() *locationMap { + return &locationMap{ + lmap: make(map[string]*time.Location), + } +} + +func (l *locationMap) getLocation(name string) (*time.Location, bool) { + l.mu.RLock() + defer l.mu.RUnlock() + result, ok := l.lmap[name] + return result, ok +} + +func (l *locationMap) setLocation(name string, value *time.Location) { + l.mu.Lock() + defer l.mu.Unlock() + l.lmap[name] = value +} + // MPPCtx is the mpp execution context type MPPCtx struct { RPCClient client.Client @@ -309,9 +336,14 @@ func buildDAG(reader *dbreader.DBReader, lockStore *lockstore.MemStore, req *cop case "System": tz = time.Local default: - tz, err = time.LoadLocation(dagReq.TimeZoneName) - if err != nil { - return nil, nil, errors.Trace(err) + var ok bool + tz, ok = globalLocationMap.getLocation(dagReq.TimeZoneName) + if !ok { + tz, err = time.LoadLocation(dagReq.TimeZoneName) + if err != nil { + return nil, nil, errors.Trace(err) + } + globalLocationMap.setLocation(dagReq.TimeZoneName, tz) } } sctx := flagsAndTzToSessionContext(dagReq.Flags, tz) diff --git a/pkg/store/mockstore/unistore/cophandler/mpp.go b/pkg/store/mockstore/unistore/cophandler/mpp.go index 77b700830761f..264dac20c5eca 100644 --- a/pkg/store/mockstore/unistore/cophandler/mpp.go +++ b/pkg/store/mockstore/unistore/cophandler/mpp.go @@ -138,10 +138,6 @@ func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error *physTblIDColIdx = numIdxCols lastCol = pb.Columns[numIdxCols-1] } - if lastCol.GetColumnId() == model.ExtraPidColID { - numIdxCols-- - lastCol = pb.Columns[numIdxCols-1] - } hdlStatus := tablecodec.HandleDefault if len(primaryColIds) == 0 { diff --git a/pkg/store/mockstore/unistore/cophandler/mpp_exec.go b/pkg/store/mockstore/unistore/cophandler/mpp_exec.go index 00adbda630d79..abed3d4b6a230 100644 --- a/pkg/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/pkg/store/mockstore/unistore/cophandler/mpp_exec.go @@ -313,7 +313,10 @@ func (e *indexScanExec) Process(key, value []byte) error { } } } - if e.physTblIDColIdx != nil { + + // If we need pid, it already filled by above loop. Because `DecodeIndexKV` func will return pid in `values`. + // The following if statement is to fill in the tid when we needed it. + if e.physTblIDColIdx != nil && *e.physTblIDColIdx >= len(values) { tblID := tablecodec.DecodeTableID(key) e.chk.AppendInt64(*e.physTblIDColIdx, tblID) } diff --git a/pkg/util/hint/hint_processor.go b/pkg/util/hint/hint_processor.go index dff5712c76f62..d76ce6c2864b9 100644 --- a/pkg/util/hint/hint_processor.go +++ b/pkg/util/hint/hint_processor.go @@ -91,7 +91,18 @@ func ExtractTableHintsFromStmtNode(node ast.Node, warnHandler hintWarnHandler) [ case *ast.InsertStmt: // check duplicated hints checkInsertStmtHintDuplicated(node, warnHandler) - return x.TableHints + result := make([]*ast.TableOptimizerHint, 0, len(x.TableHints)) + result = append(result, x.TableHints...) + if x.Select != nil { + // support statement-level hint in sub-select: "insert into t select /* ... */ ..." + // TODO: support this for Update and Delete as well + for _, h := range ExtractTableHintsFromStmtNode(x.Select, warnHandler) { + if isStmtHint(h) { + result = append(result, h) + } + } + } + return result case *ast.SetOprStmt: var result []*ast.TableOptimizerHint if x.SelectList == nil { diff --git a/pkg/util/plancache/BUILD.bazel b/pkg/util/plancache/BUILD.bazel deleted file mode 100644 index 2d729c3ac374a..0000000000000 --- a/pkg/util/plancache/BUILD.bazel +++ /dev/null @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "plancache", - srcs = ["util.go"], - importpath = "github.com/pingcap/tidb/pkg/util/plancache", - visibility = ["//visibility:public"], - deps = ["//pkg/types"], -) diff --git a/pkg/util/plancache/util.go b/pkg/util/plancache/util.go deleted file mode 100644 index f0a202573ad3e..0000000000000 --- a/pkg/util/plancache/util.go +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright 2023 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 util - -import ( - "github.com/pingcap/tidb/pkg/types" -) - -// PlanCacheMatchOpts store some property used to fetch plan from plan cache -// The structure set here is to avoid import cycle -type PlanCacheMatchOpts struct { - // paramTypes stores all parameters' FieldType, some different parameters may share same plan - ParamTypes []*types.FieldType - // limitOffsetAndCount stores all the offset and key parameters extract from limit statement - // only used for cache and pick plan with parameters in limit - LimitOffsetAndCount []uint64 - // HasSubQuery indicate whether this query has sub query - HasSubQuery bool - // StatsVersionHash is the hash value of the statistics version - StatsVersionHash uint64 - - // Below are some variables that can affect the plan - ForeignKeyChecks bool -} diff --git a/tests/integrationtest/r/globalindex/aggregate.result b/tests/integrationtest/r/globalindex/aggregate.result index 1c0103ce82509..66516451de6af 100644 --- a/tests/integrationtest/r/globalindex/aggregate.result +++ b/tests/integrationtest/r/globalindex/aggregate.result @@ -19,7 +19,7 @@ id estRows task access object operator info HashAgg 1.00 root NULL funcs:count(Column#9)->Column#4, funcs:max(Column#10)->Column#5, funcs:min(Column#11)->Column#6 └─IndexReader 1.00 root partition:p0 index:HashAgg └─HashAgg 1.00 cop[tikv] NULL funcs:count(1)->Column#9, funcs:max(globalindex__aggregate.p.id)->Column#10, funcs:min(globalindex__aggregate.p.id)->Column#11 - └─Selection 10000.00 cop[tikv] NULL in(_tidb_pid, pid0) + └─Selection 10000.00 cop[tikv] NULL in(_tidb_tid, tid0) └─IndexFullScan 10000.00 cop[tikv] table:p, index:idx(id) keep order:false, stats:pseudo select count(*), max(id), min(id) from p partition(p0) use index(idx); count(*) max(id) min(id) @@ -41,7 +41,7 @@ explain format='brief' select avg(id), max(id), min(id) from p partition(p0) use id estRows task access object operator info HashAgg 8000.00 root NULL group by:globalindex__aggregate.p.c, funcs:avg(Column#9, Column#10)->Column#4, funcs:max(Column#11)->Column#5, funcs:min(Column#12)->Column#6 └─IndexLookUp 8000.00 root partition:p0 NULL - ├─Selection(Build) 10000.00 cop[tikv] NULL in(_tidb_pid, pid0) + ├─Selection(Build) 10000.00 cop[tikv] NULL in(_tidb_tid, tid0) │ └─IndexFullScan 10000.00 cop[tikv] table:p, index:idx(id) keep order:false, stats:pseudo └─HashAgg(Probe) 8000.00 cop[tikv] NULL group by:globalindex__aggregate.p.c, funcs:count(globalindex__aggregate.p.id)->Column#9, funcs:sum(globalindex__aggregate.p.id)->Column#10, funcs:max(globalindex__aggregate.p.id)->Column#11, funcs:min(globalindex__aggregate.p.id)->Column#12 └─TableRowIDScan 10000.00 cop[tikv] table:p keep order:false, stats:pseudo diff --git a/tests/integrationtest/r/globalindex/expression_index.result b/tests/integrationtest/r/globalindex/expression_index.result index edbf260e63aed..1af9c2559a723 100644 --- a/tests/integrationtest/r/globalindex/expression_index.result +++ b/tests/integrationtest/r/globalindex/expression_index.result @@ -31,7 +31,7 @@ explain format='brief' select * from t partition(p0) use index(idx) where lower( id estRows task access object operator info Projection 3333.33 root NULL globalindex__expression_index.t.a, globalindex__expression_index.t.b └─IndexLookUp 3333.33 root partition:p0 NULL - ├─Selection(Build) 3333.33 cop[tikv] NULL in(_tidb_pid, pid0) + ├─Selection(Build) 3333.33 cop[tikv] NULL in(_tidb_tid, tid0) │ └─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(lower(`b`)) range:("c",+inf], keep order:false, stats:pseudo └─TableRowIDScan(Probe) 3333.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p0) use index(idx) where lower(b) > 'c'; diff --git a/tests/integrationtest/r/globalindex/index_join.result b/tests/integrationtest/r/globalindex/index_join.result index d49c0bf0d2e50..63e357aebb340 100644 --- a/tests/integrationtest/r/globalindex/index_join.result +++ b/tests/integrationtest/r/globalindex/index_join.result @@ -49,7 +49,7 @@ Projection 1.00 root NULL globalindex__index_join.p.id, globalindex__index_join. │ └─Selection 1.00 cop[tikv] NULL not(isnull(globalindex__index_join.t.id)) │ └─TableFullScan 1.00 cop[tikv] table:t keep order:false └─IndexLookUp(Probe) 1.00 root partition:p1 NULL - ├─Selection(Build) 1.00 cop[tikv] NULL in(_tidb_pid, pid1), not(isnull(globalindex__index_join.p.id)) + ├─Selection(Build) 1.00 cop[tikv] NULL in(_tidb_tid, tid1), not(isnull(globalindex__index_join.p.id)) │ └─IndexRangeScan 1.00 cop[tikv] table:p, index:idx(id) range: decided by [eq(globalindex__index_join.p.id, globalindex__index_join.t.id)], keep order:false └─TableRowIDScan(Probe) 1.00 cop[tikv] table:p keep order:false select * from p partition(p1) inner join t on p.id = t.id; @@ -61,7 +61,7 @@ IndexJoin 1.00 root NULL inner join, inner:IndexReader, outer key:globalindex__i │ └─Selection 1.00 cop[tikv] NULL not(isnull(globalindex__index_join.t.id)) │ └─TableFullScan 1.00 cop[tikv] table:t keep order:false └─IndexReader(Probe) 1.00 root partition:p1 index:Selection - └─Selection 1.00 cop[tikv] NULL in(_tidb_pid, pid1), not(isnull(globalindex__index_join.p.id)) + └─Selection 1.00 cop[tikv] NULL in(_tidb_tid, tid1), not(isnull(globalindex__index_join.p.id)) └─IndexRangeScan 1.00 cop[tikv] table:p, index:idx(id) range: decided by [eq(globalindex__index_join.p.id, globalindex__index_join.t.id)], keep order:false select p.id from p partition(p1) inner join t on p.id = t.id; id @@ -114,7 +114,7 @@ Projection 1.00 root NULL globalindex__index_join.p.id, globalindex__index_join. │ └─Selection 1.00 cop[tikv] NULL not(isnull(globalindex__index_join.t.id)) │ └─TableFullScan 1.00 cop[tikv] table:t keep order:false └─IndexLookUp(Probe) 1.00 root partition:p1 NULL - ├─Selection(Build) 1.00 cop[tikv] NULL in(_tidb_pid, pid1), not(isnull(globalindex__index_join.p.id)) + ├─Selection(Build) 1.00 cop[tikv] NULL in(_tidb_tid, tid1), not(isnull(globalindex__index_join.p.id)) │ └─IndexRangeScan 1.00 cop[tikv] table:p, index:idx(id) range: decided by [eq(globalindex__index_join.p.id, globalindex__index_join.t.id)], keep order:false └─TableRowIDScan(Probe) 1.00 cop[tikv] table:p keep order:false select * from p partition(p1) inner join t on p.id = t.id; @@ -126,7 +126,7 @@ IndexJoin 1.00 root NULL inner join, inner:IndexReader, outer key:globalindex__i │ └─Selection 1.00 cop[tikv] NULL not(isnull(globalindex__index_join.t.id)) │ └─TableFullScan 1.00 cop[tikv] table:t keep order:false └─IndexReader(Probe) 1.00 root partition:p1 index:Selection - └─Selection 1.00 cop[tikv] NULL in(_tidb_pid, pid1), not(isnull(globalindex__index_join.p.id)) + └─Selection 1.00 cop[tikv] NULL in(_tidb_tid, tid1), not(isnull(globalindex__index_join.p.id)) └─IndexRangeScan 1.00 cop[tikv] table:p, index:idx(id) range: decided by [eq(globalindex__index_join.p.id, globalindex__index_join.t.id)], keep order:false select p.id from p partition(p1) inner join t on p.id = t.id; id diff --git a/tests/integrationtest/r/globalindex/mem_index_lookup.result b/tests/integrationtest/r/globalindex/mem_index_lookup.result index d590c0f7d351f..4c1c9197521be 100644 --- a/tests/integrationtest/r/globalindex/mem_index_lookup.result +++ b/tests/integrationtest/r/globalindex/mem_index_lookup.result @@ -27,7 +27,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_11 3323.33 root partition:p0 NULL - ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(_tidb_pid, pid0) + ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(globalindex__mem_index_lookup.t._tidb_tid, tid0) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─TableRowIDScan_8(Probe) 3323.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p0) use index(idx1) where b <= 2; @@ -38,7 +38,7 @@ id estRows task access object operator info Projection_5 3.32 root globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b └─UnionScan_6 3.32 root eq(globalindex__mem_index_lookup.t.a, 10), le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_12 3.32 root partition:dual - ├─Selection_10(Build) 3323.33 cop[tikv] in(_tidb_pid, dual) + ├─Selection_10(Build) 3323.33 cop[tikv] in(globalindex__mem_index_lookup.t._tidb_tid, dual) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─Selection_11(Probe) 3.32 cop[tikv] eq(globalindex__mem_index_lookup.t.a, 10) └─TableRowIDScan_8 3323.33 cop[tikv] table:t keep order:false, stats:pseudo @@ -49,7 +49,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_11 3323.33 root partition:p0,p1 NULL - ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(_tidb_pid, pid0, pid1) + ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(globalindex__mem_index_lookup.t._tidb_tid, tid0, tid1) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─TableRowIDScan_8(Probe) 3323.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p0, p1) use index(idx1) where b <= 2; @@ -86,7 +86,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b, globalindex__mem_index_lookup.t.c └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_11 3323.33 root partition:p0 NULL - ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(_tidb_pid, pid0) + ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(globalindex__mem_index_lookup.t._tidb_tid, tid0) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─TableRowIDScan_8(Probe) 3323.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p0) use index(idx1) where b <= 2; @@ -97,7 +97,7 @@ id estRows task access object operator info Projection_5 0.33 root globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b, globalindex__mem_index_lookup.t.c └─UnionScan_6 0.33 root eq(globalindex__mem_index_lookup.t.a, 2010), le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_11 0.33 root partition:dual - ├─Selection_10(Build) 0.33 cop[tikv] eq(globalindex__mem_index_lookup.t.a, 2010), in(_tidb_pid, dual) + ├─Selection_10(Build) 0.33 cop[tikv] eq(globalindex__mem_index_lookup.t.a, 2010), in(globalindex__mem_index_lookup.t._tidb_tid, dual) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─TableRowIDScan_8(Probe) 0.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p1) use index(idx1) where b <= 2 and a = 2010; @@ -107,7 +107,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_lookup.t.a, globalindex__mem_index_lookup.t.b, globalindex__mem_index_lookup.t.c └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_lookup.t.b, 2) └─IndexLookUp_11 3323.33 root partition:p0,p1 NULL - ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(_tidb_pid, pid0, pid1) + ├─Selection_10(Build) 3323.33 cop[tikv] NULL in(globalindex__mem_index_lookup.t._tidb_tid, tid0, tid1) │ └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo └─TableRowIDScan_8(Probe) 3323.33 cop[tikv] table:t keep order:false, stats:pseudo select * from t partition(p0, p1) use index(idx1) where b <= 2; diff --git a/tests/integrationtest/r/globalindex/mem_index_merge.result b/tests/integrationtest/r/globalindex/mem_index_merge.result index f906a5f03d7f5..310564a52d334 100644 --- a/tests/integrationtest/r/globalindex/mem_index_merge.result +++ b/tests/integrationtest/r/globalindex/mem_index_merge.result @@ -53,7 +53,7 @@ id estRows task access object operator info Projection_5 11.00 root NULL globalindex__mem_index_merge.tpk2.a, globalindex__mem_index_merge.tpk2.b, globalindex__mem_index_merge.tpk2.c, globalindex__mem_index_merge.tpk2.d └─UnionScan_6 11.00 root NULL or(eq(globalindex__mem_index_merge.tpk2.a, 1), eq(globalindex__mem_index_merge.tpk2.b, 4)) └─IndexMerge_12 11.00 root partition:p1 type: union - ├─Selection_9(Build) 1.00 cop[tikv] NULL in(_tidb_pid, pid1) + ├─Selection_9(Build) 1.00 cop[tikv] NULL in(globalindex__mem_index_merge.tpk2._tidb_tid, tid1) │ └─IndexRangeScan_7 1.00 cop[tikv] table:tpk2, index:uidx_a(a) range:[1,1], keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 10.00 cop[tikv] table:tpk2, index:idx_bc(b, c) range:[4,4], keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 11.00 cop[tikv] table:tpk2 keep order:false, stats:pseudo @@ -66,7 +66,7 @@ id estRows task access object operator info Projection_5 1111.11 root NULL globalindex__mem_index_merge.tpk2.a, globalindex__mem_index_merge.tpk2.b, globalindex__mem_index_merge.tpk2.c, globalindex__mem_index_merge.tpk2.d └─UnionScan_6 1111.11 root NULL gt(globalindex__mem_index_merge.tpk2.a, 1), gt(globalindex__mem_index_merge.tpk2.c, 1) └─IndexMerge_12 1111.11 root partition:p1 type: intersection - ├─Selection_9(Build) 3333.33 cop[tikv] NULL in(_tidb_pid, pid1) + ├─Selection_9(Build) 3333.33 cop[tikv] NULL in(globalindex__mem_index_merge.tpk2._tidb_tid, tid1) │ └─IndexRangeScan_7 3333.33 cop[tikv] table:tpk2, index:uidx_a(a) range:(1,+inf], keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 3333.33 cop[tikv] table:tpk2, index:idx_c(c) range:(1,+inf], keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 1111.11 cop[tikv] table:tpk2 keep order:false, stats:pseudo @@ -133,7 +133,7 @@ id estRows task access object operator info Projection_5 11.00 root NULL globalindex__mem_index_merge.tpk2.a, globalindex__mem_index_merge.tpk2.b, globalindex__mem_index_merge.tpk2.c, globalindex__mem_index_merge.tpk2.d └─UnionScan_6 11.00 root NULL or(eq(globalindex__mem_index_merge.tpk2.a, 1), eq(globalindex__mem_index_merge.tpk2.b, 4)) └─IndexMerge_12 11.00 root partition:p1 type: union - ├─Selection_9(Build) 1.00 cop[tikv] NULL in(_tidb_pid, pid1) + ├─Selection_9(Build) 1.00 cop[tikv] NULL in(globalindex__mem_index_merge.tpk2._tidb_tid, tid1) │ └─IndexRangeScan_7 1.00 cop[tikv] table:tpk2, index:uidx_a(a) range:[1,1], keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 10.00 cop[tikv] table:tpk2, index:idx_bc(b, c) range:[4,4], keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 11.00 cop[tikv] table:tpk2 keep order:false, stats:pseudo @@ -146,7 +146,7 @@ id estRows task access object operator info Projection_5 1111.11 root NULL globalindex__mem_index_merge.tpk2.a, globalindex__mem_index_merge.tpk2.b, globalindex__mem_index_merge.tpk2.c, globalindex__mem_index_merge.tpk2.d └─UnionScan_6 1111.11 root NULL gt(globalindex__mem_index_merge.tpk2.a, 1), gt(globalindex__mem_index_merge.tpk2.c, 1) └─IndexMerge_12 1111.11 root partition:p1 type: intersection - ├─Selection_9(Build) 1111.11 cop[tikv] NULL gt(globalindex__mem_index_merge.tpk2.c, 1), in(_tidb_pid, pid1) + ├─Selection_9(Build) 1111.11 cop[tikv] NULL gt(globalindex__mem_index_merge.tpk2.c, 1), in(globalindex__mem_index_merge.tpk2._tidb_tid, tid1) │ └─IndexRangeScan_7 3333.33 cop[tikv] table:tpk2, index:uidx_a(a) range:(1,+inf], keep order:false, stats:pseudo ├─IndexRangeScan_10(Build) 3333.33 cop[tikv] table:tpk2, index:idx_c(c) range:(1,+inf], keep order:false, stats:pseudo └─TableRowIDScan_11(Probe) 1111.11 cop[tikv] table:tpk2 keep order:false, stats:pseudo diff --git a/tests/integrationtest/r/globalindex/mem_index_reader.result b/tests/integrationtest/r/globalindex/mem_index_reader.result index ce700261c3c6e..3578fe78b3a2e 100644 --- a/tests/integrationtest/r/globalindex/mem_index_reader.result +++ b/tests/integrationtest/r/globalindex/mem_index_reader.result @@ -26,7 +26,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_reader.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_reader.t.b, 2) └─IndexReader_10 3323.33 root partition:p0 index:Selection_9 - └─Selection_9 3323.33 cop[tikv] NULL in(_tidb_pid, pid0) + └─Selection_9 3323.33 cop[tikv] NULL in(globalindex__mem_index_reader.t._tidb_tid, tid0) └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo select b from t partition(p0) use index(idx1) where b <= 2; b @@ -36,7 +36,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_reader.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_reader.t.b, 2) └─IndexReader_10 3323.33 root partition:p0,p1 index:Selection_9 - └─Selection_9 3323.33 cop[tikv] NULL in(_tidb_pid, pid0, pid1) + └─Selection_9 3323.33 cop[tikv] NULL in(globalindex__mem_index_reader.t._tidb_tid, tid0, tid1) └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo select b from t partition(p0, p1) use index(idx1) where b <= 2; b @@ -70,7 +70,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_reader.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_reader.t.b, 2) └─IndexReader_10 3323.33 root partition:p0 index:Selection_9 - └─Selection_9 3323.33 cop[tikv] NULL in(_tidb_pid, pid0) + └─Selection_9 3323.33 cop[tikv] NULL in(globalindex__mem_index_reader.t._tidb_tid, tid0) └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo select b from t partition(p0) use index(idx1) where b <= 2; b @@ -80,7 +80,7 @@ id estRows task access object operator info Projection_5 3323.33 root NULL globalindex__mem_index_reader.t.b └─UnionScan_6 3323.33 root NULL le(globalindex__mem_index_reader.t.b, 2) └─IndexReader_10 3323.33 root partition:p0,p1 index:Selection_9 - └─Selection_9 3323.33 cop[tikv] NULL in(_tidb_pid, pid0, pid1) + └─Selection_9 3323.33 cop[tikv] NULL in(globalindex__mem_index_reader.t._tidb_tid, tid0, tid1) └─IndexRangeScan_7 3323.33 cop[tikv] table:t, index:idx1(b) range:[-inf,2], keep order:false, stats:pseudo select b from t partition(p0, p1) use index(idx1) where b <= 2; b diff --git a/tests/integrationtest/r/globalindex/misc.result b/tests/integrationtest/r/globalindex/misc.result index 57418ddc95957..ddcf665b80d8f 100644 --- a/tests/integrationtest/r/globalindex/misc.result +++ b/tests/integrationtest/r/globalindex/misc.result @@ -73,8 +73,7 @@ select * from test_t1 where a = 1; a b c 1 1 1 drop table if exists t; -create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); -alter table t add unique index (a); +create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL, unique index (a)) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); insert into t (a) values ('aaa'),('abc'),('acd'); analyze table t; select a from t partition (p0) order by a; @@ -123,11 +122,10 @@ bbc explain format = 'brief' select a from t partition (p1) order by a; id estRows task access object operator info IndexReader 3.00 root partition:p1 index:Selection -└─Selection 3.00 cop[tikv] NULL in(_tidb_pid, pid1) +└─Selection 3.00 cop[tikv] NULL in(_tidb_tid, tid1) └─IndexFullScan 3.00 cop[tikv] table:t, index:a(a) keep order:true drop table if exists t; -create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); -alter table t add unique index (a); +create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED, unique index (a)) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); insert into t (a) values ('aaa'),('abc'),('acd'); analyze table t; select a from t partition (p0) order by a; @@ -176,5 +174,5 @@ bbc explain format = 'brief' select a from t partition (p1) order by a; id estRows task access object operator info IndexReader 3.00 root partition:p1 index:Selection -└─Selection 3.00 cop[tikv] NULL in(_tidb_pid, pid1) +└─Selection 3.00 cop[tikv] NULL in(_tidb_tid, tid1) └─IndexFullScan 3.00 cop[tikv] table:t, index:a(a) keep order:true diff --git a/tests/integrationtest/t/globalindex/aggregate.test b/tests/integrationtest/t/globalindex/aggregate.test index 6d120dab7f518..02887af51847b 100644 --- a/tests/integrationtest/t/globalindex/aggregate.test +++ b/tests/integrationtest/t/globalindex/aggregate.test @@ -10,7 +10,7 @@ insert into p values (1,3), (2,3), (3,4), (4,4), (5,6), (7,9), (8,9); explain format='brief' select count(*), max(id), min(id) from p use index(idx); select count(*), max(id), min(id) from p use index(idx); ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /in\(_tidb_tid, [0-9]+\)/in(_tidb_tid, tid0)/ explain format='brief' select count(*), max(id), min(id) from p partition(p0) use index(idx); select count(*), max(id), min(id) from p partition(p0) use index(idx); @@ -18,7 +18,7 @@ explain format='brief' select avg(id), max(id), min(id) from p use index(idx) gr --sorted_result select avg(id), max(id), min(id) from p use index(idx) group by c; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /in\(_tidb_tid, [0-9]+\)/in(_tidb_tid, tid0)/ explain format='brief' select avg(id), max(id), min(id) from p partition(p0) use index(idx) group by c; select avg(id), max(id), min(id) from p partition(p0) use index(idx) group by c; diff --git a/tests/integrationtest/t/globalindex/expression_index.test b/tests/integrationtest/t/globalindex/expression_index.test index 10d28a68132a1..f3560b2d904f3 100644 --- a/tests/integrationtest/t/globalindex/expression_index.test +++ b/tests/integrationtest/t/globalindex/expression_index.test @@ -19,7 +19,7 @@ select * from t use index(idx) where lower(b) = 'c'; explain format='brief' select * from t use index(idx) where lower(b) > 'c' order by lower(b); select * from t use index(idx) where lower(b) > 'c' order by lower(b); ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /in\(_tidb_tid, [0-9]+\)/in(_tidb_tid, tid0)/ explain format='brief' select * from t partition(p0) use index(idx) where lower(b) > 'c'; select * from t partition(p0) use index(idx) where lower(b) > 'c'; diff --git a/tests/integrationtest/t/globalindex/index_join.test b/tests/integrationtest/t/globalindex/index_join.test index 621118ffc4cc0..db1357dc5b430 100644 --- a/tests/integrationtest/t/globalindex/index_join.test +++ b/tests/integrationtest/t/globalindex/index_join.test @@ -21,11 +21,11 @@ explain format='brief' select * from p inner join t on p.id = t.id; select p.id from p inner join t on p.id = t.id; --echo # TestGlobalIndexJoinSpecifiedPartition ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain format='brief' select * from p partition(p1) inner join t on p.id = t.id; select * from p partition(p1) inner join t on p.id = t.id; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain format='brief' select p.id from p partition(p1) inner join t on p.id = t.id; select p.id from p partition(p1) inner join t on p.id = t.id; @@ -49,11 +49,11 @@ explain format='brief' select * from p inner join t on p.id = t.id; select p.id from p inner join t on p.id = t.id; --echo # TestGlobalIndexJoinForClusteredSpecifiedPartition ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain format='brief' select * from p partition(p1) inner join t on p.id = t.id; select * from p partition(p1) inner join t on p.id = t.id; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain format='brief' select p.id from p partition(p1) inner join t on p.id = t.id; select p.id from p partition(p1) inner join t on p.id = t.id; diff --git a/tests/integrationtest/t/globalindex/mem_index_lookup.test b/tests/integrationtest/t/globalindex/mem_index_lookup.test index ec2c4bd1eedb4..aa7fa31d60348 100644 --- a/tests/integrationtest/t/globalindex/mem_index_lookup.test +++ b/tests/integrationtest/t/globalindex/mem_index_lookup.test @@ -18,7 +18,7 @@ explain select * from t use index(idx1) where b > 2; --sorted_result select * from t use index(idx1) where b > 2; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid0)/ explain select * from t partition(p0) use index(idx1) where b <= 2; --sorted_result select * from t partition(p0) use index(idx1) where b <= 2; @@ -27,7 +27,7 @@ explain select * from t partition(p1) use index(idx1) where b <= 2 and a = 10; --sorted_result select * from t partition(p1) use index(idx1) where b <= 2 and a = 10; ---replace_regex /in\(_tidb_pid, [0-9]+, [0-9]+\)/in(_tidb_pid, pid0, pid1)/ +--replace_regex /_tidb_tid, [0-9]+, [0-9]+\)/_tidb_tid, tid0, tid1)/ explain select * from t partition(p0, p1) use index(idx1) where b <= 2; --sorted_result select * from t partition(p0, p1) use index(idx1) where b <= 2; @@ -53,7 +53,7 @@ explain select * from t use index(idx1) where b > 2; --sorted_result select * from t use index(idx1) where b > 2; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid0)/ explain select * from t partition(p0) use index(idx1) where b <= 2; --sorted_result select * from t partition(p0) use index(idx1) where b <= 2; @@ -62,7 +62,7 @@ explain select * from t partition(p1) use index(idx1) where b <= 2 and a = 2010; --sorted_result select * from t partition(p1) use index(idx1) where b <= 2 and a = 2010; ---replace_regex /in\(_tidb_pid, [0-9]+, [0-9]+\)/in(_tidb_pid, pid0, pid1)/ +--replace_regex /_tidb_tid, [0-9]+, [0-9]+\)/_tidb_tid, tid0, tid1)/ explain select * from t partition(p0, p1) use index(idx1) where b <= 2; --sorted_result select * from t partition(p0, p1) use index(idx1) where b <= 2; diff --git a/tests/integrationtest/t/globalindex/mem_index_merge.test b/tests/integrationtest/t/globalindex/mem_index_merge.test index 5d84f3c5d9c92..f172d8f7bcd1e 100644 --- a/tests/integrationtest/t/globalindex/mem_index_merge.test +++ b/tests/integrationtest/t/globalindex/mem_index_merge.test @@ -31,13 +31,13 @@ select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 where a > 1 and c select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 where a > 0 and c > 0; --echo ## for indexMerge union with specified PARTITION ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain select /*+ use_index_merge(tpk2, uidx_a, idx_bc) */ * from tpk2 partition(p1) where a=1 or b=4; --sorted_result select /*+ use_index_merge(tpk2, uidx_a, idx_bc) */ * from tpk2 partition(p1) where a=1 or b=4; --echo ## for indexMerge intersection with specified PARTITION ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 partition(p1) where a > 1 and c > 1; --sorted_result select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 partition(p1) where a > 1 and c > 1; @@ -80,13 +80,13 @@ select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 where a > 1 and c select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 where a > 0 and c > 0; --echo ## for indexMerge union with specified PARTITION ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain select /*+ use_index_merge(tpk2, uidx_a, idx_bc) */ * from tpk2 partition(p1) where a=1 or b=4; --sorted_result select /*+ use_index_merge(tpk2, uidx_a, idx_bc) */ * from tpk2 partition(p1) where a=1 or b=4; --echo ## for indexMerge intersection with specified PARTITION ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid1)/ explain select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 partition(p1) where a > 1 and c > 1; --sorted_result select /*+ use_index_merge(tpk2, uidx_a, idx_c) */ * from tpk2 partition(p1) where a > 1 and c > 1; diff --git a/tests/integrationtest/t/globalindex/mem_index_reader.test b/tests/integrationtest/t/globalindex/mem_index_reader.test index b243205bfffc8..83080f8a8f2d7 100644 --- a/tests/integrationtest/t/globalindex/mem_index_reader.test +++ b/tests/integrationtest/t/globalindex/mem_index_reader.test @@ -18,12 +18,12 @@ explain select b from t use index(idx1) where b > 2; --sorted_result select b from t use index(idx1) where b > 2; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid0)/ explain select b from t partition(p0) use index(idx1) where b <= 2; --sorted_result select b from t partition(p0) use index(idx1) where b <= 2; ---replace_regex /in\(_tidb_pid, [0-9]+, [0-9]+\)/in(_tidb_pid, pid0, pid1)/ +--replace_regex /_tidb_tid, [0-9]+, [0-9]+\)/_tidb_tid, tid0, tid1)/ explain select b from t partition(p0, p1) use index(idx1) where b <= 2; --sorted_result select b from t partition(p0, p1) use index(idx1) where b <= 2; @@ -48,12 +48,12 @@ explain select b from t use index(idx1) where b > 2; --sorted_result select b from t use index(idx1) where b > 2; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid0)/ +--replace_regex /_tidb_tid, [0-9]+\)/_tidb_tid, tid0)/ explain select b from t partition(p0) use index(idx1) where b <= 2; --sorted_result select b from t partition(p0) use index(idx1) where b <= 2; ---replace_regex /in\(_tidb_pid, [0-9]+, [0-9]+\)/in(_tidb_pid, pid0, pid1)/ +--replace_regex /_tidb_tid, [0-9]+, [0-9]+\)/_tidb_tid, tid0, tid1)/ explain select b from t partition(p0, p1) use index(idx1) where b <= 2; --sorted_result select b from t partition(p0, p1) use index(idx1) where b <= 2; diff --git a/tests/integrationtest/t/globalindex/misc.test b/tests/integrationtest/t/globalindex/misc.test index 6b40dcc828c94..015385a3c5ea9 100644 --- a/tests/integrationtest/t/globalindex/misc.test +++ b/tests/integrationtest/t/globalindex/misc.test @@ -81,8 +81,7 @@ select * from test_t1 where a = 1; ## Test generated column with global index drop table if exists t; ## Test for virtual generated column with global index -create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); -alter table t add unique index (a); +create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL, unique index (a)) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); insert into t (a) values ('aaa'),('abc'),('acd'); analyze table t; select a from t partition (p0) order by a; @@ -100,13 +99,12 @@ select a from t order by a; select * from t where a = 'bbc'; select a from t partition (p0) order by a; select a from t partition (p1) order by a; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /in\(_tidb_tid, [0-9]+\)/in(_tidb_tid, tid1)/ explain format = 'brief' select a from t partition (p1) order by a; drop table if exists t; ## Test for stored generated column with global index -create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); -alter table t add unique index (a); +create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED, unique index (a)) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d')); insert into t (a) values ('aaa'),('abc'),('acd'); analyze table t; select a from t partition (p0) order by a; @@ -124,6 +122,6 @@ select a from t order by a; select * from t where a = 'bbc'; select a from t partition (p0) order by a; select a from t partition (p1) order by a; ---replace_regex /in\(_tidb_pid, [0-9]+\)/in(_tidb_pid, pid1)/ +--replace_regex /in\(_tidb_tid, [0-9]+\)/in(_tidb_tid, tid1)/ explain format = 'brief' select a from t partition (p1) order by a;