Skip to content

Commit

Permalink
Merge branch 'master' into fix-parser
Browse files Browse the repository at this point in the history
  • Loading branch information
Defined2014 authored Aug 22, 2022
2 parents 9a2f114 + e0da196 commit 7c5cd79
Show file tree
Hide file tree
Showing 25 changed files with 673 additions and 89 deletions.
71 changes: 71 additions & 0 deletions docs/design/2022-08-02-stats-lru-cache.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
# Proposal: Stats LRU Cache

- Author(s): [Yisaer](https://github.com/yisaer)
- Tracking Issue: https://github.com/pingcap/tidb/issues/34052

## Abstract

This proposes a design of how to maintain stats cache by lru according to memory usage

## Background

Previously, tidb maintained the all the indices' stats and some columns' stats which is needed during query.
As the maintained stats grows, the total memory usage of the stats will increase and makes tidb server OOM.
Thus we use lru to maintain stats cache in order to keep memory safe.

### Goal

- Use LRU to maintain the stats cache in memory
- Keep the total memory usage of stats cache under the quota
- Support loading stats back into memory when tidb server needs it

### Non-Goals

- Considering the stats cache are in memory, we don't provide changing stats cache into LRU without restarting

## Proposal

### Stats Cache Interface

We will provide a Stats Cache Interface which is implemented by LRU Cache and Map Cache.
If the tidb server didn't enable Stats LRU Cache, it will use Map Cache by default. Also, we will provide config and global session variable to control whether enable Stats LRU Cache and the capacity of it.

```go
// statsCacheInner is the interface to manage the statsCache, it can be implemented by map, lru cache or other structures.
type statsCacheInner interface {
Get(int64) (*statistics.Table, bool)
Put(int64, *statistics.Table)
Del(int64)
Cost() int64
Len() int
SetCapacity(int64)
}
```

### Stats LRU Cache Policy

For column or index stats, we maintained following data structure for stats:

- `Histogram`
- `TopN`
- `CMSketch`

And we will also maintain status for each column and index stats in order to indicate its stats loading status like following:

- `AllLoaded`
- `OnlyCMSEvcited`
- `OnlyHistRemained`
- `AllEvicted`

When the column or index stats load all data structures into memory, the status will be `AllLoaded`.
When the Stats LRU Cache memory usage exceeds the quota, the LRU Cache will select one column or index stats to evict the data structures by following rules to reduce the memory usage:

- If the status is `AllLoaded`, it will discard the `CMSketch` and the status will be changed into `OnlyCMSEvcited`
- If the status is `OnlyCMSEvcited`, it will discard the `TopN` and the status will be changed into `OnlyHistRemained`
- If the status is `OnlyHistRemained`, it will discard the `Histogram` and the status will be changed into `AllEvicted`

### Sync Stats Compatibility

Previously tidb server has Sync Stats and asynchronously Loading Histograms in order to load column stats into memory during query.
As Stats LRU Cache Policy may evict index stats in memory, we also need Sync Stats and asynchronously Loading Histograms to support loading index stats according to its loading status to keep compatible.
During the query optimization, tidb server will collect the used columns and indices, if their stats are not fully loaded, tidb-server will try to load their stats back into the memory.
8 changes: 8 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1296,7 +1296,15 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
ExecRetryCount: a.retryCount,
IsExplicitTxn: sessVars.TxnCtx.IsExplicit,
IsWriteCacheTable: stmtCtx.WaitLockLeaseTime > 0,
IsSyncStatsFailed: stmtCtx.IsSyncStatsFailed,
}
failpoint.Inject("assertSyncStatsFailed", func(val failpoint.Value) {
if val.(bool) {
if !slowItems.IsSyncStatsFailed {
panic("isSyncStatsFailed should be true")
}
}
})
if a.retryCount > 0 {
slowItems.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime)
}
Expand Down
1 change: 1 addition & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1925,6 +1925,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.EnableOptimizeTrace = false
sc.OptimizeTracer = nil
sc.OptimizerCETrace = nil
sc.IsSyncStatsFailed = false

sc.SysdateIsNow = ctx.GetSessionVars().SysdateIsNow

Expand Down
18 changes: 17 additions & 1 deletion executor/temporary_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,29 @@ func TestTemporaryTableNoNetwork(t *testing.T) {
t.Run("global", func(t *testing.T) {
assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) {
tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows")
tk.MustExec("begin")
})
})

t.Run("global create and then truncate", func(t *testing.T) {
assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) {
tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows")
tk.MustExec("truncate table tmp_t")
tk.MustExec("begin")
})
})

t.Run("local", func(t *testing.T) {
assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) {
tk.MustExec("create temporary table tmp_t (id int primary key, a int, b int, index(a))")
tk.MustExec("begin")
})
})

t.Run("local and create table inside txn", func(t *testing.T) {
assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) {
tk.MustExec("begin")
tk.MustExec("create temporary table tmp_t (id int primary key, a int, b int, index(a))")
})
})
}
Expand All @@ -61,7 +78,6 @@ func assertTemporaryTableNoNetwork(t *testing.T, createTable func(*testkit.TestK
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy"))
}()

tk.MustExec("begin")
tk.MustExec("insert into tmp_t values (1, 1, 1)")
tk.MustExec("insert into tmp_t values (2, 2, 2)")

Expand Down
19 changes: 19 additions & 0 deletions infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -697,6 +697,10 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i
})
b.is.sortedTablesBuckets[bucketIdx] = sortedTbls

if tblInfo.TempTableType != model.TempTableNone {
b.addTemporaryTable(tableID)
}

newTbl, ok := b.is.TableByID(tableID)
if ok {
dbInfo.Tables = append(dbInfo.Tables, newTbl.Meta())
Expand Down Expand Up @@ -750,6 +754,11 @@ func (b *Builder) applyDropTable(dbInfo *model.DBInfo, tableID int64, affected [
// Remove the table in sorted table slice.
b.is.sortedTablesBuckets[bucketIdx] = append(sortedTbls[0:idx], sortedTbls[idx+1:]...)

// Remove the table in temporaryTables
if b.is.temporaryTableIDs != nil {
delete(b.is.temporaryTableIDs, tableID)
}

// The old DBInfo still holds a reference to old table info, we need to remove it.
for i, tblInfo := range dbInfo.Tables {
if tblInfo.ID == tableID {
Expand Down Expand Up @@ -895,10 +904,20 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF
schTbls.tables[t.Name.L] = tbl
sortedTbls := b.is.sortedTablesBuckets[tableBucketIdx(t.ID)]
b.is.sortedTablesBuckets[tableBucketIdx(t.ID)] = append(sortedTbls, tbl)
if tblInfo := tbl.Meta(); tblInfo.TempTableType != model.TempTableNone {
b.addTemporaryTable(tblInfo.ID)
}
}
return nil
}

func (b *Builder) addTemporaryTable(tblID int64) {
if b.is.temporaryTableIDs == nil {
b.is.temporaryTableIDs = make(map[int64]struct{})
}
b.is.temporaryTableIDs[tblID] = struct{}{}
}

type virtualTableDriver struct {
*model.DBInfo
TableFromMeta tableFromMetaFunc
Expand Down
15 changes: 15 additions & 0 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,8 @@ type InfoSchema interface {
AllPlacementBundles() []*placement.Bundle
// AllPlacementPolicies returns all placement policies
AllPlacementPolicies() []*model.PolicyInfo
// HasTemporaryTable returns whether information schema has temporary table
HasTemporaryTable() bool
}

type sortedTables []table.Table
Expand Down Expand Up @@ -94,6 +96,9 @@ type infoSchema struct {
// sortedTablesBuckets is a slice of sortedTables, a table's bucket index is (tableID % bucketCount).
sortedTablesBuckets []sortedTables

// temporaryTables stores the temporary table ids
temporaryTableIDs map[int64]struct{}

// schemaMetaVersion is the version of schema, and we should check version when change schema.
schemaMetaVersion int64
}
Expand Down Expand Up @@ -302,6 +307,11 @@ func (is *infoSchema) FindTableByPartitionID(partitionID int64) (table.Table, *m
return nil, nil, nil
}

// HasTemporaryTable returns whether information schema has temporary table
func (is *infoSchema) HasTemporaryTable() bool {
return len(is.temporaryTableIDs) != 0
}

func (is *infoSchema) Clone() (result []*model.DBInfo) {
for _, v := range is.schemaMap {
result = append(result, v.dbInfo.Clone())
Expand Down Expand Up @@ -567,3 +577,8 @@ func (ts *TemporaryTableAttachedInfoSchema) SchemaByTable(tableInfo *model.Table

return ts.InfoSchema.SchemaByTable(tableInfo)
}

// HasTemporaryTable returns whether information schema has temporary table
func (ts *TemporaryTableAttachedInfoSchema) HasTemporaryTable() bool {
return ts.LocalTemporaryTables.Count() > 0 || ts.InfoSchema.HasTemporaryTable()
}
124 changes: 124 additions & 0 deletions infoschema/infoschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,130 @@ func genGlobalID(store kv.Storage) (int64, error) {
return globalID, errors.Trace(err)
}

func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

is := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema)
require.False(t, is.HasTemporaryTable())
db, ok := is.SchemaByName(model.NewCIStr("test"))
require.True(t, ok)

doChange := func(changes ...func(m *meta.Meta, builder *infoschema.Builder)) infoschema.InfoSchema {
builder, err := infoschema.NewBuilder(store, nil).InitWithDBInfos([]*model.DBInfo{db}, nil, is.SchemaMetaVersion())
require.NoError(t, err)
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL)
err = kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
m := meta.NewMeta(txn)
for _, change := range changes {
change(m, builder)
}
return nil
})
require.NoError(t, err)
return builder.Build()
}

createGlobalTemporaryTableChange := func(tblID int64) func(m *meta.Meta, builder *infoschema.Builder) {
return func(m *meta.Meta, builder *infoschema.Builder) {
err := m.CreateTableOrView(db.ID, &model.TableInfo{
ID: tblID,
TempTableType: model.TempTableGlobal,
State: model.StatePublic,
})
require.NoError(t, err)
_, err = builder.ApplyDiff(m, &model.SchemaDiff{Type: model.ActionCreateTable, SchemaID: db.ID, TableID: tblID})
require.NoError(t, err)
}
}

dropTableChange := func(tblID int64) func(m *meta.Meta, builder *infoschema.Builder) {
return func(m *meta.Meta, builder *infoschema.Builder) {
err := m.DropTableOrView(db.ID, tblID)
require.NoError(t, err)
_, err = builder.ApplyDiff(m, &model.SchemaDiff{Type: model.ActionDropTable, SchemaID: db.ID, TableID: tblID})
require.NoError(t, err)
}
}

truncateGlobalTemporaryTableChange := func(tblID, newTblID int64) func(m *meta.Meta, builder *infoschema.Builder) {
return func(m *meta.Meta, builder *infoschema.Builder) {
err := m.DropTableOrView(db.ID, tblID)
require.NoError(t, err)

err = m.CreateTableOrView(db.ID, &model.TableInfo{
ID: newTblID,
TempTableType: model.TempTableGlobal,
State: model.StatePublic,
})
require.NoError(t, err)
_, err = builder.ApplyDiff(m, &model.SchemaDiff{Type: model.ActionTruncateTable, SchemaID: db.ID, OldTableID: tblID, TableID: newTblID})
require.NoError(t, err)
}
}

alterTableChange := func(tblID int64) func(m *meta.Meta, builder *infoschema.Builder) {
return func(m *meta.Meta, builder *infoschema.Builder) {
_, err := builder.ApplyDiff(m, &model.SchemaDiff{Type: model.ActionAddColumn, SchemaID: db.ID, TableID: tblID})
require.NoError(t, err)
}
}

// create table
tbID, err := genGlobalID(store)
require.NoError(t, err)
newIS := doChange(
createGlobalTemporaryTableChange(tbID),
)
require.True(t, newIS.HasTemporaryTable())

// full load
newDB, ok := newIS.SchemaByName(model.NewCIStr("test"))
require.True(t, ok)
builder, err := infoschema.NewBuilder(store, nil).InitWithDBInfos([]*model.DBInfo{newDB}, newIS.AllPlacementPolicies(), newIS.SchemaMetaVersion())
require.NoError(t, err)
require.True(t, builder.Build().HasTemporaryTable())

// create and then drop
tbID, err = genGlobalID(store)
require.NoError(t, err)
require.False(t, doChange(
createGlobalTemporaryTableChange(tbID),
dropTableChange(tbID),
).HasTemporaryTable())

// create and then alter
tbID, err = genGlobalID(store)
require.NoError(t, err)
require.True(t, doChange(
createGlobalTemporaryTableChange(tbID),
alterTableChange(tbID),
).HasTemporaryTable())

// create and truncate
tbID, err = genGlobalID(store)
require.NoError(t, err)
newTbID, err := genGlobalID(store)
require.NoError(t, err)
require.True(t, doChange(
createGlobalTemporaryTableChange(tbID),
truncateGlobalTemporaryTableChange(tbID, newTbID),
).HasTemporaryTable())

// create two and drop one
tbID, err = genGlobalID(store)
require.NoError(t, err)
tbID2, err := genGlobalID(store)
require.NoError(t, err)
require.True(t, doChange(
createGlobalTemporaryTableChange(tbID),
createGlobalTemporaryTableChange(tbID2),
dropTableChange(tbID),
).HasTemporaryTable())
}

func TestBuildBundle(t *testing.T) {
store := testkit.CreateMockStore(t)

Expand Down
Loading

0 comments on commit 7c5cd79

Please sign in to comment.