domain,infoschema: put the @@tidb_schema_cache_size variable into effect (#53424)

ref pingcap/tidb#50959
This commit is contained in:
tiancaiamao
2024-05-27 12:46:19 +08:00
committed by GitHub
parent 0ac2ad0252
commit d24fdb702e
9 changed files with 71 additions and 6 deletions

View File

@ -454,6 +454,7 @@ go_test(
"//pkg/util/syncutil",
"//pkg/util/tableutil",
"//pkg/util/topsql/state",
"@com_github_docker_go_units//:go-units",
"@com_github_gorilla_mux//:mux",
"@com_github_hashicorp_go_version//:go-version",
"@com_github_pingcap_errors//:errors",

View File

@ -19,6 +19,7 @@ import (
"testing"
"time"
"github.com/docker/go-units"
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
@ -292,7 +293,7 @@ func TestShowStatusSnapshot(t *testing.T) {
UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment)
tk.MustExec(updateSafePoint)
for _, cacheSize := range []int{1024, 0} {
for _, cacheSize := range []int{units.GiB, 0} {
tk.MustExec("set @@global.tidb_schema_cache_size = ?", cacheSize)
tk.MustExec("create table t (a int);")
snapshotTime := time.Now()

View File

@ -20,6 +20,7 @@ import (
"testing"
"time"
"github.com/docker/go-units"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config"
@ -893,7 +894,7 @@ func TestSetTransactionInfoSchema(t *testing.T) {
UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment)
tk.MustExec(updateSafePoint)
for _, cacheSize := range []int{1024, 0} {
for _, cacheSize := range []int{units.GiB, 0} {
tk.MustExec("set @@global.tidb_schema_cache_size = ?", cacheSize)
testSetTransactionInfoSchema(t, tk)
}

View File

@ -977,14 +977,19 @@ func RegisterVirtualTable(dbInfo *model.DBInfo, tableFromMeta tableFromMetaFunc)
// NewBuilder creates a new Builder with a Handle.
func NewBuilder(r autoid.Requirement, factory func() (pools.Resource, error), infoData *Data) *Builder {
return &Builder{
enableV2: variable.SchemaCacheSize.Load() > 0,
builder := &Builder{
Requirement: r,
infoschemaV2: NewInfoSchemaV2(r, infoData),
dirtyDB: make(map[string]bool),
factory: factory,
infoData: infoData,
}
schemaCacheSize := variable.SchemaCacheSize.Load()
if schemaCacheSize > 0 {
infoData.tableCache.SetCapacity(uint64(schemaCacheSize))
builder.enableV2 = true
}
return builder
}
func tableBucketIdx(tableID int64) int {

View File

@ -162,6 +162,11 @@ func NewData() *Data {
return ret
}
// CacheCapacity is exported for testing.
func (isd *Data) CacheCapacity() uint64 {
return isd.tableCache.Capacity()
}
func (isd *Data) add(item tableItem, tbl table.Table) {
isd.byID.Set(item)
isd.byName.Set(item)
@ -424,6 +429,7 @@ func (is *infoschemaV2) SchemaTableInfos(schema model.CIStr) []*model.TableInfo
return getTableInfoList(tables)
}
retry:
dbInfo, ok := is.SchemaByName(schema)
if !ok {
return nil
@ -438,6 +444,13 @@ func (is *infoschemaV2) SchemaTableInfos(schema model.CIStr) []*model.TableInfo
if meta.ErrDBNotExists.Equal(err) {
return nil
}
// Flashback statement could cause such kind of error.
// In theory that error should be handled in the lower layer, like client-go.
// But it's not done, so we retry here.
if strings.Contains(err.Error(), "in flashback progress") {
time.Sleep(200 * time.Millisecond)
goto retry
}
// TODO: error could happen, so do not panic!
panic(err)
}
@ -650,6 +663,7 @@ retry:
func loadTableInfo(r autoid.Requirement, infoData *Data, tblID, dbID int64, ts uint64, schemaVersion int64) (table.Table, error) {
// Try to avoid repeated concurrency loading.
res, err, _ := loadTableSF.Do(fmt.Sprintf("%d-%d-%d", dbID, tblID, schemaVersion), func() (any, error) {
retry:
snapshot := r.Store().GetSnapshot(kv.NewVersion(ts))
// Using the KV timeout read feature to address the issue of potential DDL lease expiration when
// the meta region leader is slow.
@ -657,8 +671,15 @@ func loadTableInfo(r autoid.Requirement, infoData *Data, tblID, dbID int64, ts u
m := meta.NewSnapshotMeta(snapshot)
tblInfo, err := m.GetTable(dbID, tblID)
if err != nil {
// Flashback statement could cause such kind of error.
// In theory that error should be handled in the lower layer, like client-go.
// But it's not done, so we retry here.
if strings.Contains(err.Error(), "in flashback progress") {
time.Sleep(200 * time.Millisecond)
goto retry
}
// TODO load table panic!!!
panic(err)
}

View File

@ -75,6 +75,12 @@ func (s *Sieve[K, V]) SetCapacity(capacity uint64) {
s.capacity = capacity
}
func (s *Sieve[K, V]) Capacity() uint64 {
s.mu.Lock()
defer s.mu.Unlock()
return s.capacity
}
func (s *Sieve[K, V]) Set(key K, value V) {
s.mu.Lock()
defer s.mu.Unlock()

View File

@ -8,11 +8,13 @@ go_test(
"v2_test.go",
],
flaky = True,
shard_count = 3,
deps = [
"//pkg/domain",
"//pkg/infoschema",
"//pkg/parser/auth",
"//pkg/parser/model",
"//pkg/sessionctx/variable",
"//pkg/table",
"//pkg/testkit",
"//pkg/testkit/testsetup",

View File

@ -15,12 +15,14 @@
package infoschemav2test
import (
"strconv"
"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/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/stretchr/testify/require"
@ -147,3 +149,29 @@ PARTITION p5 VALUES LESS THAN (1980))`)
pid = pi.GetPartitionIDByName("p3")
require.Equal(t, pid, ntbl.Meta().ID)
}
func TestTiDBSchemaCacheSizeVariable(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
dom.Reload() // need this to trigger infoschema rebuild to reset capacity
is := dom.InfoSchema()
ok, raw := infoschema.IsV2(is)
if ok {
val := variable.SchemaCacheSize.Load()
tk.MustQuery("select @@global.tidb_schema_cache_size").CheckContain(strconv.FormatInt(val, 10))
// On start, the capacity might not be set correctly because infoschema have not load global variable yet.
// cap := raw.Data.CacheCapacity()
// require.Equal(t, cap, uint64(val))
}
tk.MustExec("set @@global.tidb_schema_cache_size = 32 * 1024 * 1024")
tk.MustQuery("select @@global.tidb_schema_cache_size").CheckContain("33554432")
require.Equal(t, variable.SchemaCacheSize.Load(), int64(33554432))
tk.MustExec("create table trigger_reload (id int)") // need to trigger infoschema rebuild to reset capacity
is = dom.InfoSchema()
ok, raw = infoschema.IsV2(is)
require.True(t, ok)
require.Equal(t, raw.Data.CacheCapacity(), uint64(33554432))
}

View File

@ -120,7 +120,7 @@ func (tk *TestKit) RefreshSession() {
if intest.InTest {
if rand.Intn(10) >= 3 { // 70% chance to run infoschema v2
tk.MustExec("set @@global.tidb_schema_cache_size = 1024")
tk.MustExec("set @@global.tidb_schema_cache_size = 1024 * 1024 * 1024")
}
}