Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into issue40910
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent committed Feb 1, 2023
2 parents 715c42e + bdc2f1b commit ffdbed2
Show file tree
Hide file tree
Showing 4 changed files with 117 additions and 42 deletions.
17 changes: 13 additions & 4 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -7617,10 +7617,14 @@ func (d *ddl) CreateResourceGroup(ctx sessionctx.Context, stmt *ast.CreateResour
return err
}
}
if !stmt.IfNotExists {
if _, ok := d.GetInfoSchemaWithInterceptor(ctx).ResourceGroupByName(groupName); ok {
return infoschema.ErrResourceGroupExists.GenWithStackByArgs(groupName)

if _, ok := d.GetInfoSchemaWithInterceptor(ctx).ResourceGroupByName(groupName); ok {
if stmt.IfNotExists {
err = infoschema.ErrResourceGroupExists.GenWithStackByArgs(groupName)
ctx.GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return infoschema.ErrResourceGroupExists.GenWithStackByArgs(groupName)
}

if groupName.L == defaultResourceGroupName {
Expand Down Expand Up @@ -7710,7 +7714,12 @@ func (d *ddl) AlterResourceGroup(ctx sessionctx.Context, stmt *ast.AlterResource
// Check group existence.
group, ok := is.ResourceGroupByName(groupName)
if !ok {
return infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(groupName)
err := infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(groupName)
if stmt.IfExists {
ctx.GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
}
newGroupInfo, err := buildResourceGroup(group, stmt.ResourceGroupOptionList)
if err != nil {
Expand Down
18 changes: 18 additions & 0 deletions ddl/resource_group_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,17 @@ func TestResourceGroupBasic(t *testing.T) {
g := testResourceGroupNameFromIS(t, tk.Session(), "x")
checkFunc(g)

// test create if not exists
tk.MustExec("create resource group if not exists x " +
"RRU_PER_SEC=10000 " +
"WRU_PER_SEC=20000")
// Check the resource group is not changed
g = testResourceGroupNameFromIS(t, tk.Session(), "x")
checkFunc(g)
// Check warning message
res := tk.MustQuery("show warnings")
res.Check(testkit.Rows("Note 8248 Resource group 'x' already exists"))

tk.MustExec("set global tidb_enable_resource_control = DEFAULT")
tk.MustGetErrCode("alter resource group x "+
"RRU_PER_SEC=2000 "+
Expand All @@ -91,6 +102,13 @@ func TestResourceGroupBasic(t *testing.T) {
re.Equal(uint64(2000), g.RRURate)
re.Equal(uint64(3000), g.WRURate)

tk.MustExec("alter resource group if exists not_exists " +
"RRU_PER_SEC=2000 " +
"WRU_PER_SEC=3000")
// Check warning message
res = tk.MustQuery("show warnings")
res.Check(testkit.Rows("Note 8249 Unknown resource group 'not_exists'"))

tk.MustQuery("select * from information_schema.resource_groups where group_name = 'x'").Check(testkit.Rows(strconv.FormatInt(g.ID, 10) + " x 2000 3000"))

tk.MustExec("drop resource group x")
Expand Down
30 changes: 30 additions & 0 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -365,3 +365,33 @@ PARTITION p0 VALUES LESS THAN (6)
require.NotNil(t, jsTable.Partitions["p0"])
require.NotNil(t, jsTable.Partitions["global"])
}

func TestDumpHistoricalStatsFallback(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 0")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec(`CREATE TABLE t (a int, b int, index idx(b))
PARTITION BY RANGE ( a ) (
PARTITION p0 VALUES LESS THAN (6)
)`)
// dump historical stats
tk.MustExec("analyze table t")
is := dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
require.NotNil(t, tbl)

// dump historical stats
hsWorker := dom.GetHistoricalStatsWorker()
tblID := hsWorker.GetOneHistoricalStatsTable()
// assert no historical stats task generated
require.Equal(t, tblID, int64(-1))
tk.MustExec("set global tidb_enable_historical_stats = 1")
h := dom.StatsHandle()
jt, err := h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), oracle.GoTimeToTS(time.Now()))
require.NoError(t, err)
require.NotNil(t, jt)
require.False(t, jt.IsHistoricalStats)
}
94 changes: 56 additions & 38 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"bytes"
"compress/gzip"
"encoding/json"
"fmt"
"io"
"time"

Expand All @@ -40,15 +39,16 @@ import (

// JSONTable is used for dumping statistics.
type JSONTable struct {
DatabaseName string `json:"database_name"`
TableName string `json:"table_name"`
Columns map[string]*jsonColumn `json:"columns"`
Indices map[string]*jsonColumn `json:"indices"`
ExtStats []*jsonExtendedStats `json:"ext_stats"`
Count int64 `json:"count"`
ModifyCount int64 `json:"modify_count"`
Partitions map[string]*JSONTable `json:"partitions"`
Version uint64 `json:"version"`
IsHistoricalStats bool `json:"is_historical_stats"`
DatabaseName string `json:"database_name"`
TableName string `json:"table_name"`
Columns map[string]*jsonColumn `json:"columns"`
Indices map[string]*jsonColumn `json:"indices"`
ExtStats []*jsonExtendedStats `json:"ext_stats"`
Count int64 `json:"count"`
ModifyCount int64 `json:"modify_count"`
Partitions map[string]*JSONTable `json:"partitions"`
Version uint64 `json:"version"`
}

type jsonExtendedStats struct {
Expand Down Expand Up @@ -142,45 +142,44 @@ var (

// DumpHistoricalStatsBySnapshot dumped json tables from mysql.stats_meta_history and mysql.stats_history
func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64) (jt *JSONTable, err error) {
historicalStatsEnabled, err := h.CheckHistoricalStatsEnable()
if err != nil {
return nil, errors.Errorf("check %v failed: %v", variable.TiDBEnableHistoricalStats, err)
}
if !historicalStatsEnabled {
return nil, errors.Errorf("%v should be enabled", variable.TiDBEnableHistoricalStats)
}

defer func() {
if err == nil {
dumpHistoricalStatsSuccessCounter.Inc()
} else {
dumpHistoricalStatsFailedCounter.Inc()
}
}()

pi := tableInfo.GetPartitionInfo()
if pi == nil {
return h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot)
return h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, tableInfo.ID, snapshot)
}
jsonTbl := &JSONTable{
DatabaseName: dbName,
TableName: tableInfo.Name.L,
Partitions: make(map[string]*JSONTable, len(pi.Definitions)),
}
for _, def := range pi.Definitions {
tbl, err := h.tableHistoricalStatsToJSON(def.ID, snapshot)
tbl, err := h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, def.ID, snapshot)
if err != nil {
return nil, errors.Trace(err)
}
if tbl == nil {
continue
}
jsonTbl.Partitions[def.Name.L] = tbl
}
h.mu.Lock()
isDynamicMode := variable.PartitionPruneMode(h.mu.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic
h.mu.Unlock()
if isDynamicMode {
tbl, err := h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot)
if err != nil {
logutil.BgLogger().Warn("dump global historical stats failed",
zap.Int64("table-id", tableInfo.ID),
zap.String("table-name", tableInfo.Name.String()))
} else if tbl != nil {
jsonTbl.Partitions["global"] = tbl
}
tbl, err := h.getTableHistoricalStatsToJSONWithFallback(dbName, tableInfo, tableInfo.ID, snapshot)
if err != nil {
return nil, err
}
// dump its global-stats if existed
if tbl != nil {
jsonTbl.Partitions["global"] = tbl
}
return jsonTbl, nil
}
Expand Down Expand Up @@ -250,11 +249,23 @@ func GenJSONTableFromStats(dbName string, tableInfo *model.TableInfo, tbl *stati
return jsonTbl, nil
}

func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) (*JSONTable, error) {
reader, err := h.getGlobalStatsReader(0)
// getTableHistoricalStatsToJSONWithFallback try to get table historical stats, if not exit, directly fallback to the latest stats
func (h *Handle) getTableHistoricalStatsToJSONWithFallback(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) {
jt, exist, err := h.tableHistoricalStatsToJSON(physicalID, snapshot)
if err != nil {
return nil, err
}
if !exist {
return h.tableStatsToJSON(dbName, tableInfo, physicalID, 0)
}
return jt, nil
}

func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) (*JSONTable, bool, error) {
reader, err := h.getGlobalStatsReader(0)
if err != nil {
return nil, false, err
}
defer func() {
err1 := h.releaseGlobalStatsReader(reader)
if err == nil && err1 != nil {
Expand All @@ -265,45 +276,52 @@ func (h *Handle) tableHistoricalStatsToJSON(physicalID int64, snapshot uint64) (
// get meta version
rows, _, err := reader.Read("select distinct version from mysql.stats_meta_history where table_id = %? and version <= %? order by version desc limit 1", physicalID, snapshot)
if err != nil {
return nil, errors.AddStack(err)
return nil, false, errors.AddStack(err)
}
if len(rows) < 1 {
return nil, fmt.Errorf("failed to get records of stats_meta_history for table_id = %v, snapshot = %v", physicalID, snapshot)
logutil.BgLogger().Warn("failed to get records of stats_meta_history",
zap.Int64("table-id", physicalID),
zap.Uint64("snapshotTS", snapshot))
return nil, false, nil
}
statsMetaVersion := rows[0].GetInt64(0)
// get stats meta
rows, _, err = reader.Read("select modify_count, count from mysql.stats_meta_history where table_id = %? and version = %?", physicalID, statsMetaVersion)
if err != nil {
return nil, errors.AddStack(err)
return nil, false, errors.AddStack(err)
}
modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1)

// get stats version
rows, _, err = reader.Read("select distinct version from mysql.stats_history where table_id = %? and version <= %? order by version desc limit 1", physicalID, snapshot)
if err != nil {
return nil, errors.AddStack(err)
return nil, false, errors.AddStack(err)
}
if len(rows) < 1 {
return nil, fmt.Errorf("failed to get record of stats_history for table_id = %v, snapshot = %v", physicalID, snapshot)
logutil.BgLogger().Warn("failed to get record of stats_history",
zap.Int64("table-id", physicalID),
zap.Uint64("snapshotTS", snapshot))
return nil, false, nil
}
statsVersion := rows[0].GetInt64(0)

// get stats
rows, _, err = reader.Read("select stats_data from mysql.stats_history where table_id = %? and version = %? order by seq_no", physicalID, statsVersion)
if err != nil {
return nil, errors.AddStack(err)
return nil, false, errors.AddStack(err)
}
blocks := make([][]byte, 0)
for _, row := range rows {
blocks = append(blocks, row.GetBytes(0))
}
jsonTbl, err := BlocksToJSONTable(blocks)
if err != nil {
return nil, errors.AddStack(err)
return nil, false, errors.AddStack(err)
}
jsonTbl.Count = count
jsonTbl.ModifyCount = modifyCount
return jsonTbl, nil
jsonTbl.IsHistoricalStats = true
return jsonTbl, true, nil
}

func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) {
Expand Down

0 comments on commit ffdbed2

Please sign in to comment.