domain,infoschema: put the @@tidb_schema_cache_size variable into effect (#53424)
ref pingcap/tidb#50959
This commit is contained in:
@ -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",
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user