diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index cfe76b016f254..9df8319aeef66 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -981,8 +981,6 @@ func (b *Builder) Build() InfoSchema { b.updateInfoSchemaBundles(b.infoSchema) if b.enableV2 { b.infoschemaV2.ts = math.MaxUint64 // TODO: should be the correct TS - b.infoschemaV2.r = b.Requirement - b.infoschemaV2.Data = b.infoData b.infoschemaV2.schemaVersion = b.infoSchema.SchemaMetaVersion() return &b.infoschemaV2 } @@ -1258,6 +1256,8 @@ func NewBuilder(r autoid.Requirement, factory func() (pools.Resource, error), in schemaMap: map[string]*schemaTables{}, sortedTablesBuckets: make([]sortedTables, bucketCount), }, + Data: infoData, + r: r, }, dirtyDB: make(map[string]bool), factory: factory, diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index e23c400c5ea68..e362be15dbbbb 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -239,13 +239,6 @@ func search(bt *btree.BTreeG[tableItem], schemaVersion int64, end tableItem, mat } func (is *infoschemaV2) TableByID(id int64) (val table.Table, ok bool) { - if isTableVirtual(id) { - // Don't store the virtual table in the tableCache, because when cache missing - // we can't refill it from tikv. - // TODO: returns the correct result. - return nil, false - } - // Get from the cache. key := tableCacheKey{id, is.schemaVersion} tbl, found := is.tableCache.Get(key) @@ -260,6 +253,14 @@ func (is *infoschemaV2) TableByID(id int64) (val table.Table, ok bool) { return nil, false } + if isTableVirtual(id) { + if schTbls, exist := is.Data.specials[itm.dbName]; exist { + val, ok = schTbls.tables[itm.tableName] + return + } + return nil, false + } + // Maybe the table is evicted? need to reload. ret, err := loadTableInfo(is.r, is.Data, id, itm.dbID, is.ts, is.schemaVersion) if err == nil { @@ -309,6 +310,10 @@ func (is *infoschemaV2) TableByName(schema, tbl model.CIStr) (t table.Table, err } func (is *infoschemaV2) SchemaByName(schema model.CIStr) (val *model.DBInfo, ok bool) { + if isSpecialDB(schema.L) { + return is.Data.specials[schema.L].dbInfo, true + } + var dbInfo model.DBInfo dbInfo.Name = schema is.Data.schemaMap.Descend(schemaItem{dbInfo: &dbInfo, schemaVersion: math.MaxInt64}, func(item schemaItem) bool { @@ -332,6 +337,9 @@ func (is *infoschemaV2) AllSchemas() (schemas []*model.DBInfo) { schemas = append(schemas, item.dbInfo) return true }) + for _, sc := range is.Data.specials { + schemas = append(schemas, sc.dbInfo) + } return } @@ -341,6 +349,11 @@ func (is *infoschemaV2) SchemaMetaVersion() int64 { func (is *infoschemaV2) SchemaExists(schema model.CIStr) bool { var ok bool + if isSpecialDB(schema.L) { + _, ok = is.Data.specials[schema.L] + return ok + } + // TODO: support different version is.Data.schemaMap.Scan(func(item schemaItem) bool { if item.dbInfo.Name.L == schema.L { @@ -364,6 +377,16 @@ func (is *infoschemaV2) TableExists(schema, table model.CIStr) bool { func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) { var ok bool var dbInfo *model.DBInfo + if isTableVirtual(id) { + for _, st := range is.Data.specials { + if st.dbInfo.ID == id { + return st.dbInfo, true + } + } + // Something wrong? + return nil, false + } + is.Data.schemaMap.Scan(func(item schemaItem) bool { if item.dbInfo.ID == id { ok = true @@ -376,6 +399,15 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) { } func (is *infoschemaV2) SchemaTables(schema model.CIStr) (tables []table.Table) { + if isSpecialDB(schema.L) { + schTbls := is.Data.specials[schema.L] + tables := make([]table.Table, 0, len(schTbls.tables)) + for _, tbl := range schTbls.tables { + tables = append(tables, tbl) + } + return tables + } + dbInfo, ok := is.SchemaByName(schema) if !ok { return diff --git a/pkg/infoschema/test/infoschemav2test/BUILD.bazel b/pkg/infoschema/test/infoschemav2test/BUILD.bazel new file mode 100644 index 0000000000000..3dbafe928efe2 --- /dev/null +++ b/pkg/infoschema/test/infoschemav2test/BUILD.bazel @@ -0,0 +1,20 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "infoschemav2test_test", + timeout = "short", + srcs = [ + "main_test.go", + "v2_test.go", + ], + flaky = True, + deps = [ + "//pkg/domain", + "//pkg/infoschema", + "//pkg/parser/auth", + "//pkg/testkit", + "//pkg/testkit/testsetup", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/infoschema/test/infoschemav2test/main_test.go b/pkg/infoschema/test/infoschemav2test/main_test.go new file mode 100644 index 0000000000000..0df733a61bc8f --- /dev/null +++ b/pkg/infoschema/test/infoschemav2test/main_test.go @@ -0,0 +1,34 @@ +// 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 infoschemav2test + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), + goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/pkg/infoschema/test/infoschemav2test/v2_test.go b/pkg/infoschema/test/infoschemav2test/v2_test.go new file mode 100644 index 0000000000000..0186c99013cb3 --- /dev/null +++ b/pkg/infoschema/test/infoschemav2test/v2_test.go @@ -0,0 +1,67 @@ +// 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 infoschemav2test + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/stretchr/testify/require" +) + +func TestSpecialSchemas(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil, nil)) + tk.MustExec("use test") + + tk.MustQuery("select @@global.tidb_schema_cache_size;").Check(testkit.Rows("0")) + tk.MustExec("set @@global.tidb_schema_cache_size = 1024;") + tk.MustQuery("select @@global.tidb_schema_cache_size;").Check(testkit.Rows("1024")) + tk.MustExec("create table t (id int);") + is := domain.GetDomain(tk.Session()).InfoSchema() + require.True(t, infoschema.IsV2(is)) + + tk.MustQuery("show databases;").Check(testkit.Rows( + "INFORMATION_SCHEMA", "METRICS_SCHEMA", "PERFORMANCE_SCHEMA", "mysql", "sys", "test")) + tk.MustExec("use information_schema;") + tk.MustQuery("show tables;").MultiCheckContain([]string{ + "ANALYZE_STATUS", + "ATTRIBUTES", + "CHARACTER_SETS", + "COLLATIONS", + "COLUMNS", + "COLUMN_PRIVILEGES", + "COLUMN_STATISTICS", + "VIEWS"}) + tk.MustQuery("show create table tables;").MultiCheckContain([]string{ + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `TABLE_TYPE`, + }) + + tk.ExecToErr("drop database information_schema;") + tk.ExecToErr("drop table views;") + + tk.MustExec("use metrics_schema;") + tk.MustQuery("show tables;").CheckContain("uptime") + tk.MustQuery("show create table uptime;").CheckContain("time") + + tk.MustExec("set @@global.tidb_schema_cache_size = default;") +}