90 Star 491 Fork 151

平凯星辰(北京)科技有限公司/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
gc.go 5.06 KB
一键复制 编辑 原始数据 按行查看 历史
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package statistics
import (
"fmt"
"time"
"github.com/cznic/mathutil"
"github.com/juju/errors"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/util/sqlexec"
"golang.org/x/net/context"
)
// GCStats will garbage collect the useless stats info. For dropped tables, we will first update their version so that
// other tidb could know that table is deleted.
func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error {
// To make sure that all the deleted tables' schema and stats info have been acknowledged to all tidb,
// we only garbage collect version before 10 lease.
lease := mathutil.MaxInt64(int64(h.Lease), int64(ddlLease))
offset := oracle.ComposeTS(10*lease, 0)
if h.PrevLastVersion < offset {
return nil
}
sql := fmt.Sprintf("select table_id from mysql.stats_meta where version < %d", h.PrevLastVersion-offset)
rows, _, err := h.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(h.ctx, sql)
if err != nil {
return errors.Trace(err)
}
for _, row := range rows {
if err := h.gcTableStats(is, row.GetInt64(0)); err != nil {
return errors.Trace(err)
}
}
return nil
}
func (h *Handle) gcTableStats(is infoschema.InfoSchema, tableID int64) error {
sql := fmt.Sprintf("select is_index, hist_id from mysql.stats_histograms where table_id = %d", tableID)
rows, _, err := h.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(h.ctx, sql)
if err != nil {
return errors.Trace(err)
}
// The table has already been deleted in stats and acknowledged to all tidb,
// we can safely remove the meta info now.
if len(rows) == 0 {
sql := fmt.Sprintf("delete from mysql.stats_meta where table_id = %d", tableID)
_, _, err := h.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(h.ctx, sql)
return errors.Trace(err)
}
tbl, ok := is.TableByID(tableID)
if !ok {
return errors.Trace(h.DeleteTableStatsFromKV(tableID))
}
tblInfo := tbl.Meta()
for _, row := range rows {
isIndex, histID := row.GetInt64(0), row.GetInt64(1)
find := false
if isIndex == 1 {
for _, idx := range tblInfo.Indices {
if idx.ID == histID {
find = true
break
}
}
} else {
for _, col := range tblInfo.Columns {
if col.ID == histID {
find = true
break
}
}
}
if !find {
if err := h.deleteHistStatsFromKV(tblInfo.ID, histID, int(isIndex)); err != nil {
return errors.Trace(err)
}
}
}
return nil
}
// deleteHistStatsFromKV deletes all records about a column or an index and updates version.
func (h *Handle) deleteHistStatsFromKV(tableID int64, histID int64, isIndex int) error {
exec := h.ctx.(sqlexec.SQLExecutor)
_, err := exec.Execute(context.Background(), "begin")
if err != nil {
return errors.Trace(err)
}
// First of all, we update the version. If this table doesn't exist, it won't have any problem. Because we cannot delete anything.
_, err = exec.Execute(context.Background(), fmt.Sprintf("update mysql.stats_meta set version = %d where table_id = %d ", h.ctx.Txn().StartTS(), tableID))
if err != nil {
return errors.Trace(err)
}
// delete histogram meta
_, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_histograms where table_id = %d and hist_id = %d and is_index = %d", tableID, histID, isIndex))
if err != nil {
return errors.Trace(err)
}
// delete all buckets
_, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d and hist_id = %d and is_index = %d", tableID, histID, isIndex))
if err != nil {
return errors.Trace(err)
}
_, err = exec.Execute(context.Background(), "commit")
return errors.Trace(err)
}
// DeleteTableStatsFromKV deletes table statistics from kv.
func (h *Handle) DeleteTableStatsFromKV(id int64) error {
exec := h.ctx.(sqlexec.SQLExecutor)
_, err := exec.Execute(context.Background(), "begin")
if err != nil {
return errors.Trace(err)
}
// We only update the version so that other tidb will know that this table is deleted.
sql := fmt.Sprintf("update mysql.stats_meta set version = %d where table_id = %d ", h.ctx.Txn().StartTS(), id)
_, err = exec.Execute(context.Background(), sql)
if err != nil {
return errors.Trace(err)
}
_, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_histograms where table_id = %d", id))
if err != nil {
return errors.Trace(err)
}
_, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d", id))
if err != nil {
return errors.Trace(err)
}
_, err = exec.Execute(context.Background(), "commit")
return errors.Trace(err)
}
Loading...
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v2.0.6

搜索帮助

0d507c66 1850385 C8b1a773 1850385