Skip to content

Commit

Permalink
domain,infoschema: avoid v1/v2 switch when loading snapshot infoschema (
Browse files Browse the repository at this point in the history
#55129)

close #55114
  • Loading branch information
tiancaiamao authored Aug 3, 2024
1 parent 7e4608f commit 52b4c8a
Show file tree
Hide file tree
Showing 7 changed files with 69 additions and 42 deletions.
4 changes: 3 additions & 1 deletion pkg/ddl/placement_policy_ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/dbterror"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -119,7 +120,8 @@ func TestPlacementPolicyInUse(t *testing.T) {
t4.State = model.StatePublic
db1.Deprecated.Tables = append(db1.Deprecated.Tables, t4)

builder, err := infoschema.NewBuilder(dom, nil, infoschema.NewData()).InitWithDBInfos(
builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0)
err = builder.InitWithDBInfos(
[]*model.DBInfo{db1, db2, dbP},
[]*model.PolicyInfo{p1, p2, p3, p4, p5},
nil,
Expand Down
45 changes: 27 additions & 18 deletions pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,7 @@ func (do *Domain) EtcdClient() *clientv3.Client {
// 3. currentSchemaVersion(before loading)
// 4. the changed table IDs if it is not full load
// 5. an error if any
func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, int64, *transaction.RelatedSchemaChange, error) {
func (do *Domain) loadInfoSchema(startTS uint64, isSnapshot bool) (infoschema.InfoSchema, bool, int64, *transaction.RelatedSchemaChange, error) {
beginTime := time.Now()
defer func() {
infoschema_metrics.LoadSchemaDurationTotal.Observe(time.Since(beginTime).Seconds())
Expand All @@ -299,7 +299,6 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i
schemaTs = 0
}

enableV2 := variable.SchemaCacheSize.Load() > 0
if is := do.infoCache.GetByVersion(neededSchemaVersion); is != nil {
isV2, raw := infoschema.IsV2(is)
if isV2 {
Expand All @@ -315,18 +314,18 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i
// the insert method check if schemaTs is zero
do.infoCache.Insert(is, schemaTs)

if enableV2 == isV2 {
return is, true, 0, nil, nil
}
return is, true, 0, nil, nil
}

var isV1V2Switch bool
var oldIsV2 bool
enableV2 := variable.SchemaCacheSize.Load() > 0
currentSchemaVersion := int64(0)
if oldInfoSchema := do.infoCache.GetLatest(); oldInfoSchema != nil {
currentSchemaVersion = oldInfoSchema.SchemaMetaVersion()
isV2, _ := infoschema.IsV2(oldInfoSchema)
isV1V2Switch = enableV2 != isV2
oldIsV2, _ = infoschema.IsV2(oldInfoSchema)
}
useV2, isV1V2Switch := shouldUseV2(enableV2, oldIsV2, isSnapshot)
builder := infoschema.NewBuilder(do, do.sysFacHack, do.infoCache.Data, useV2)

// TODO: tryLoadSchemaDiffs has potential risks of failure. And it becomes worse in history reading cases.
// It is only kept because there is no alternative diff/partial loading solution.
Expand All @@ -336,8 +335,8 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i
// 3. There are less 100 diffs.
// 4. No regenerated schema diff.
startTime := time.Now()
if currentSchemaVersion != 0 && neededSchemaVersion > currentSchemaVersion && neededSchemaVersion-currentSchemaVersion < LoadSchemaDiffVersionGapThreshold {
is, relatedChanges, diffTypes, err := do.tryLoadSchemaDiffs(m, currentSchemaVersion, neededSchemaVersion, startTS)
if !isV1V2Switch && currentSchemaVersion != 0 && neededSchemaVersion > currentSchemaVersion && neededSchemaVersion-currentSchemaVersion < LoadSchemaDiffVersionGapThreshold {
is, relatedChanges, diffTypes, err := do.tryLoadSchemaDiffs(builder, m, currentSchemaVersion, neededSchemaVersion, startTS)
if err == nil {
infoschema_metrics.LoadSchemaDurationLoadDiff.Observe(time.Since(startTime).Seconds())
isV2, _ := infoschema.IsV2(is)
Expand Down Expand Up @@ -373,11 +372,11 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i
}
infoschema_metrics.LoadSchemaDurationLoadAll.Observe(time.Since(startTime).Seconds())

newISBuilder, err := infoschema.NewBuilder(do, do.sysFacHack, do.infoCache.Data).InitWithDBInfos(schemas, policies, resourceGroups, neededSchemaVersion)
err = builder.InitWithDBInfos(schemas, policies, resourceGroups, neededSchemaVersion)
if err != nil {
return nil, false, currentSchemaVersion, nil, err
}
is := newISBuilder.Build(startTS)
is := builder.Build(startTS)
isV2, _ := infoschema.IsV2(is)
logutil.BgLogger().Info("full load InfoSchema success",
zap.Bool("isV2", isV2),
Expand Down Expand Up @@ -534,11 +533,21 @@ func (*Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, don
done <- nil
}

// shouldUseV2 decides whether to use infoschema v2.
// When loading snapshot, infoschema should keep the same as before to avoid v1/v2 switch.
// Otherwise, it is decided by enabledV2.
func shouldUseV2(enableV2 bool, oldIsV2 bool, isSnapshot bool) (useV2 bool, isV1V2Switch bool) {
if isSnapshot {
return oldIsV2, false
}
return enableV2, enableV2 != oldIsV2
}

// tryLoadSchemaDiffs tries to only load latest schema changes.
// Return true if the schema is loaded successfully.
// Return false if the schema can not be loaded by schema diff, then we need to do full load.
// The second returned value is the delta updated table and partition IDs.
func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64, startTS uint64) (infoschema.InfoSchema, *transaction.RelatedSchemaChange, []string, error) {
func (do *Domain) tryLoadSchemaDiffs(builder *infoschema.Builder, m *meta.Meta, usedVersion, newVersion int64, startTS uint64) (infoschema.InfoSchema, *transaction.RelatedSchemaChange, []string, error) {
var diffs []*model.SchemaDiff
for usedVersion < newVersion {
usedVersion++
Expand Down Expand Up @@ -573,17 +582,17 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64
}
})

builder, err := infoschema.NewBuilder(do, do.sysFacHack, do.infoCache.Data).InitWithOldInfoSchema(do.infoCache.GetLatest())
err := builder.InitWithOldInfoSchema(do.infoCache.GetLatest())
if err != nil {
return nil, nil, nil, errors.Trace(err)
}

builder.WithStore(do.store).SetDeltaUpdateBundles()
phyTblIDs := make([]int64, 0, len(diffs))
actions := make([]uint64, 0, len(diffs))
diffTypes := make([]string, 0, len(diffs))
for _, diff := range diffs {
if diff.RegenerateSchemaMap {
do.infoCache.Data = infoschema.NewData()
return nil, nil, nil, errors.Errorf("Meets a schema diff with RegenerateSchemaMap flag")
}
ids, err := builder.ApplyDiff(m, diff)
Expand Down Expand Up @@ -626,7 +635,7 @@ func (do *Domain) GetSnapshotInfoSchema(snapshotTS uint64) (infoschema.InfoSchem
if is := do.infoCache.GetBySnapshotTS(snapshotTS); is != nil {
return is, nil
}
is, _, _, _, err := do.loadInfoSchema(snapshotTS)
is, _, _, _, err := do.loadInfoSchema(snapshotTS, true)
infoschema_metrics.LoadSchemaCounterSnapshot.Inc()
return is, err
}
Expand Down Expand Up @@ -727,12 +736,12 @@ func (do *Domain) Reload() error {
}

version := ver.Ver
is, hitCache, oldSchemaVersion, changes, err := do.loadInfoSchema(version)
is, hitCache, oldSchemaVersion, changes, err := do.loadInfoSchema(version, false)
if err != nil {
if version = getFlashbackStartTSFromErrorMsg(err); version != 0 {
// use the latest available version to create domain
version--
is, hitCache, oldSchemaVersion, changes, err = do.loadInfoSchema(version)
is, hitCache, oldSchemaVersion, changes, err = do.loadInfoSchema(version, false)
}
}
if err != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/slow_query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,8 @@ func parseLog(retriever *slowQueryRetriever, sctx sessionctx.Context, reader *bu

func newSlowQueryRetriever() (*slowQueryRetriever, error) {
data := infoschema.NewData()
newISBuilder, err := infoschema.NewBuilder(nil, nil, data).InitWithDBInfos(nil, nil, nil, 0)
newISBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0)
err := newISBuilder.InitWithDBInfos(nil, nil, nil, 0)
if err != nil {
return nil, err
}
Expand Down
10 changes: 7 additions & 3 deletions pkg/executor/stmtsummary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/mock"
Expand All @@ -32,7 +33,8 @@ import (

func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) {
data := infoschema.NewData()
infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil, data).InitWithDBInfos(nil, nil, nil, 0)
infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0)
err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0)
require.NoError(t, err)
infoSchema := infoSchemaBuilder.Build(math.MaxUint64)
table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummary))
Expand Down Expand Up @@ -76,7 +78,8 @@ func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) {

func TestStmtSummaryRetriverV2_TableStatementsSummaryEvicted(t *testing.T) {
data := infoschema.NewData()
infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil, data).InitWithDBInfos(nil, nil, nil, 0)
infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0)
err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0)
require.NoError(t, err)
infoSchema := infoSchemaBuilder.Build(math.MaxUint64)
table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryEvicted))
Expand Down Expand Up @@ -155,7 +158,8 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) {
stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3"))

data := infoschema.NewData()
infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil, data).InitWithDBInfos(nil, nil, nil, 0)
infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0)
err = infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0)
require.NoError(t, err)
infoSchema := infoSchemaBuilder.Build(math.MaxUint64)
table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryHistory))
Expand Down
18 changes: 9 additions & 9 deletions pkg/infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -795,12 +795,12 @@ func (b *Builder) Build(schemaTS uint64) InfoSchema {
}

// InitWithOldInfoSchema initializes an empty new InfoSchema by copies all the data from old InfoSchema.
func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) (*Builder, error) {
func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) error {
// Do not mix infoschema v1 and infoschema v2 building, this can simplify the logic.
// If we want to build infoschema v2, but the old infoschema is v1, just return error to trigger a full load.
isV2, _ := IsV2(oldSchema)
if b.enableV2 != isV2 {
return nil, errors.Errorf("builder's (v2=%v) infoschema mismatch, return error to trigger full reload", b.enableV2)
return errors.Errorf("builder's (v2=%v) infoschema mismatch, return error to trigger full reload", b.enableV2)
}

if schemaV2, ok := oldSchema.(*infoschemaV2); ok {
Expand All @@ -818,7 +818,7 @@ func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) (*Builder, error)
b.infoSchema.referredForeignKeyMap = maps.Clone(oldIS.referredForeignKeyMap)

copy(b.infoSchema.sortedTablesBuckets, oldIS.sortedTablesBuckets)
return b, nil
return nil
}

// getSchemaAndCopyIfNecessary creates a new schemaTables instance when a table in the database has changed.
Expand Down Expand Up @@ -860,7 +860,7 @@ func (b *Builder) sortAllTablesByID() {
}

// InitWithDBInfos initializes an empty new InfoSchema with a slice of DBInfo, all placement rules, and schema version.
func (b *Builder) InitWithDBInfos(dbInfos []*model.DBInfo, policies []*model.PolicyInfo, resourceGroups []*model.ResourceGroupInfo, schemaVersion int64) (*Builder, error) {
func (b *Builder) InitWithDBInfos(dbInfos []*model.DBInfo, policies []*model.PolicyInfo, resourceGroups []*model.ResourceGroupInfo, schemaVersion int64) error {
info := b.infoSchema
info.schemaMetaVersion = schemaVersion

Expand Down Expand Up @@ -890,18 +890,18 @@ func (b *Builder) InitWithDBInfos(dbInfos []*model.DBInfo, policies []*model.Pol
for _, di := range dbInfos {
err := b.createSchemaTablesForDB(di, b.tableFromMeta, schemaVersion)
if err != nil {
return nil, errors.Trace(err)
return errors.Trace(err)
}
}

err := b.initVirtualTables(schemaVersion)
if err != nil {
return nil, err
return err
}

b.sortAllTablesByID()

return b, nil
return nil
}

func (b *Builder) tableFromMeta(alloc autoid.Allocators, tblInfo *model.TableInfo) (table.Table, error) {
Expand Down Expand Up @@ -1008,18 +1008,18 @@ 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 {
func NewBuilder(r autoid.Requirement, factory func() (pools.Resource, error), infoData *Data, useV2 bool) *Builder {
builder := &Builder{
Requirement: r,
infoschemaV2: NewInfoSchemaV2(r, infoData),
dirtyDB: make(map[string]bool),
factory: factory,
infoData: infoData,
enableV2: useV2,
}
schemaCacheSize := variable.SchemaCacheSize.Load()
if schemaCacheSize > 0 {
infoData.tableCache.SetCapacity(schemaCacheSize)
builder.enableV2 = true
}
return builder
}
Expand Down
24 changes: 16 additions & 8 deletions pkg/infoschema/infoschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,8 @@ func TestBasic(t *testing.T) {
internal.AddDB(t, re.Store(), dbInfo)
internal.AddTable(t, re.Store(), dbInfo, tblInfo)

builder, err := infoschema.NewBuilder(re, nil, infoschema.NewData()).InitWithDBInfos(dbInfos, nil, nil, 1)
builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0)
err = builder.InitWithDBInfos(dbInfos, nil, nil, 1)
require.NoError(t, err)

txn, err := re.Store().Begin()
Expand Down Expand Up @@ -279,7 +280,8 @@ func TestInfoTables(t *testing.T) {
require.NoError(t, err)
}()

builder, err := infoschema.NewBuilder(re, nil, infoschema.NewData()).InitWithDBInfos(nil, nil, nil, 0)
builder := infoschema.NewBuilder(re, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos(nil, nil, nil, 0)
require.NoError(t, err)
is := builder.Build(math.MaxUint64)

Expand Down Expand Up @@ -343,7 +345,8 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) {
dbInfo.Deprecated.Tables = []*model.TableInfo{}
dbInfos := []*model.DBInfo{dbInfo}
data := infoschema.NewData()
builder, err := infoschema.NewBuilder(re, nil, data).InitWithDBInfos(dbInfos, nil, nil, 1)
builder := infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos(dbInfos, nil, nil, 1)
require.NoError(t, err)
is := builder.Build(math.MaxUint64)
require.False(t, is.HasTemporaryTable())
Expand All @@ -363,7 +366,8 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) {
err := kv.RunInNewTxn(ctx, re.Store(), true, func(ctx context.Context, txn kv.Transaction) error {
m := meta.NewMeta(txn)
for _, change := range changes {
builder, err := infoschema.NewBuilder(re, nil, data).InitWithOldInfoSchema(curIs)
builder = infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithOldInfoSchema(curIs)
require.NoError(t, err)
change(m, builder)
curIs = builder.Build(math.MaxUint64)
Expand Down Expand Up @@ -446,7 +450,8 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) {
require.NoError(t, err)
newDB.Deprecated.Tables = tblInfos
require.True(t, ok)
builder, err = infoschema.NewBuilder(re, nil, data).InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.AllResourceGroups(), newIS.SchemaMetaVersion())
builder = infoschema.NewBuilder(re, nil, data, variable.SchemaCacheSize.Load() > 0)
err = builder.InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.AllResourceGroups(), newIS.SchemaMetaVersion())
require.NoError(t, err)
require.True(t, builder.Build(math.MaxUint64).HasTemporaryTable())

Expand Down Expand Up @@ -576,7 +581,8 @@ func TestBuildBundle(t *testing.T) {
db.Deprecated.Tables, err = is.SchemaTableInfos(context.Background(), db.Name)
require.NoError(t, err)
}
builder, err := infoschema.NewBuilder(dom, nil, infoschema.NewData()).InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion())
builder := infoschema.NewBuilder(dom, nil, infoschema.NewData(), variable.SchemaCacheSize.Load() > 0)
err = builder.InitWithDBInfos([]*model.DBInfo{db}, is.AllPlacementPolicies(), is.AllResourceGroups(), is.SchemaMetaVersion())
require.NoError(t, err)
is2 := builder.Build(math.MaxUint64)
assertBundle(is2, tbl1.Meta().ID, tb1Bundle)
Expand Down Expand Up @@ -999,7 +1005,8 @@ func (tc *infoschemaTestContext) createSchema() {
internal.AddDB(tc.t, tc.re.Store(), dbInfo)
tc.dbInfo = dbInfo
// init infoschema
builder, err := infoschema.NewBuilder(tc.re, nil, tc.data).InitWithDBInfos([]*model.DBInfo{}, nil, nil, 1)
builder := infoschema.NewBuilder(tc.re, nil, tc.data, variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos([]*model.DBInfo{}, nil, nil, 1)
require.NoError(tc.t, err)
tc.is = builder.Build(math.MaxUint64)
}
Expand Down Expand Up @@ -1183,7 +1190,8 @@ func (tc *infoschemaTestContext) applyDiffAndCheck(diff *model.SchemaDiff, check
txn, err := tc.re.Store().Begin()
require.NoError(tc.t, err)

builder, err := infoschema.NewBuilder(tc.re, nil, tc.data).InitWithOldInfoSchema(tc.is)
builder := infoschema.NewBuilder(tc.re, nil, tc.data, variable.SchemaCacheSize.Load() > 0)
err = builder.InitWithOldInfoSchema(tc.is)
require.NoError(tc.t, err)
// applyDiff
_, err = builder.ApplyDiff(meta.NewMeta(txn), diff)
Expand Down
7 changes: 5 additions & 2 deletions pkg/infoschema/infoschema_v2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -127,7 +128,8 @@ func TestMisc(t *testing.T) {
r.Store().Close()
}()

builder, err := NewBuilder(r, nil, NewData()).InitWithDBInfos(nil, nil, nil, 1)
builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos(nil, nil, nil, 1)
require.NoError(t, err)
is := builder.Build(math.MaxUint64)
require.Len(t, is.AllResourceGroups(), 0)
Expand Down Expand Up @@ -249,7 +251,8 @@ func TestBundles(t *testing.T) {

schemaName := model.NewCIStr("testDB")
tableName := model.NewCIStr("test")
builder, err := NewBuilder(r, nil, NewData()).InitWithDBInfos(nil, nil, nil, 1)
builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0)
err := builder.InitWithDBInfos(nil, nil, nil, 1)
require.NoError(t, err)
is := builder.Build(math.MaxUint64)
require.Equal(t, 2, len(is.AllSchemas()))
Expand Down

0 comments on commit 52b4c8a

Please sign in to comment.