Skip to content

Commit

Permalink
Merge branch 'master' into fix-create-resource-group
Browse files Browse the repository at this point in the history
  • Loading branch information
glorv authored Jan 31, 2023
2 parents 342c419 + 4fd710c commit 0b27925
Show file tree
Hide file tree
Showing 35 changed files with 785 additions and 85 deletions.
11 changes: 11 additions & 0 deletions bindinfo/bind_record.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,17 @@ type BindRecord struct {
Bindings []Binding
}

// Copy get the copy of bindRecord
func (br *BindRecord) Copy() *BindRecord {
nbr := &BindRecord{
OriginalSQL: br.OriginalSQL,
Db: br.Db,
}
nbr.Bindings = make([]Binding, len(br.Bindings))
copy(nbr.Bindings, br.Bindings)
return nbr
}

// HasEnabledBinding checks if there are any enabled bindings in bind record.
func (br *BindRecord) HasEnabledBinding() bool {
for _, binding := range br.Bindings {
Expand Down
3 changes: 2 additions & 1 deletion ddl/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,8 @@ func isFlashbackSupportedDDLAction(action model.ActionType) bool {
switch action {
case model.ActionSetTiFlashReplica, model.ActionUpdateTiFlashReplicaStatus, model.ActionAlterPlacementPolicy,
model.ActionAlterTablePlacement, model.ActionAlterTablePartitionPlacement, model.ActionCreatePlacementPolicy,
model.ActionDropPlacementPolicy, model.ActionModifySchemaDefaultPlacement:
model.ActionDropPlacementPolicy, model.ActionModifySchemaDefaultPlacement,
model.ActionAlterTableAttributes, model.ActionAlterTablePartitionAttributes:
return false
default:
return true
Expand Down
19 changes: 18 additions & 1 deletion domain/historical_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,13 @@ package domain

import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)

var (
Expand All @@ -35,7 +38,21 @@ type HistoricalStatsWorker struct {

// SendTblToDumpHistoricalStats send tableID to worker to dump historical stats
func (w *HistoricalStatsWorker) SendTblToDumpHistoricalStats(tableID int64) {
w.tblCH <- tableID
send := enableDumpHistoricalStats.Load()
failpoint.Inject("sendHistoricalStats", func(val failpoint.Value) {
if val.(bool) {
send = true
}
})
if !send {
return
}
select {
case w.tblCH <- tableID:
return
default:
logutil.BgLogger().Warn("discard dump historical stats task", zap.Int64("table-id", tableID))
}
}

// DumpHistoricalStats dump stats by given tableID
Expand Down
1 change: 0 additions & 1 deletion executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,6 @@ go_library(
"//util/servermemorylimit",
"//util/set",
"//util/size",
"//util/slice",
"//util/sqlexec",
"//util/stmtsummary",
"//util/stringutil",
Expand Down
59 changes: 59 additions & 0 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
Expand All @@ -30,6 +31,8 @@ import (
)

func TestRecordHistoryStatsAfterAnalyze(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)

tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -150,6 +153,8 @@ func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) {
}

func TestGCHistoryStatsAfterDropTable(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
Expand All @@ -174,6 +179,7 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) {
tableInfo.Meta().ID)).Check(testkit.Rows("1"))
// drop the table and gc stats
tk.MustExec("drop table t")
is = dom.InfoSchema()
h.GCStats(is, 0)

// assert stats_history tables delete the record of dropped table
Expand All @@ -183,7 +189,56 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) {
tableInfo.Meta().ID)).Check(testkit.Rows("0"))
}

func TestAssertHistoricalStatsAfterAlterTable(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10),c int, KEY `idx` (`c`))")
tk.MustExec("analyze table test.t")
is := dom.InfoSchema()
tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
// dump historical stats
h := dom.StatsHandle()
hsWorker := dom.GetHistoricalStatsWorker()
tblID := hsWorker.GetOneHistoricalStatsTable()
err = hsWorker.DumpHistoricalStats(tblID, h)
require.Nil(t, err)

time.Sleep(1 * time.Second)
snapshot := oracle.GoTimeToTS(time.Now())
jsTable, err := h.DumpHistoricalStatsBySnapshot("test", tableInfo.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
require.NotEqual(t, jsTable.Version, uint64(0))
originVersion := jsTable.Version

// assert historical stats non-change after drop column
tk.MustExec("alter table t drop column b")
h.GCStats(is, 0)
snapshot = oracle.GoTimeToTS(time.Now())
jsTable, err = h.DumpHistoricalStatsBySnapshot("test", tableInfo.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
require.Equal(t, jsTable.Version, originVersion)

// assert historical stats non-change after drop index
tk.MustExec("alter table t drop index idx")
h.GCStats(is, 0)
snapshot = oracle.GoTimeToTS(time.Now())
jsTable, err = h.DumpHistoricalStatsBySnapshot("test", tableInfo.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
require.Equal(t, jsTable.Version, originVersion)
}

func TestGCOutdatedHistoryStats(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
Expand Down Expand Up @@ -219,6 +274,8 @@ func TestGCOutdatedHistoryStats(t *testing.T) {
}

func TestPartitionTableHistoricalStats(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
Expand Down Expand Up @@ -246,6 +303,8 @@ PARTITION p0 VALUES LESS THAN (6)
}

func TestDumpHistoricalStatsByTable(t *testing.T) {
failpoint.Enable("github.com/pingcap/tidb/domain/sendHistoricalStats", "return(true)")
defer failpoint.Disable("github.com/pingcap/tidb/domain/sendHistoricalStats")
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
Expand Down
60 changes: 60 additions & 0 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,3 +806,63 @@ func TestIssue38323(t *testing.T) {
tk.MustExec("set @a = 1;")
tk.MustQuery("execute stmt using @a, @a").Check(tk.MustQuery("explain select * from t where 1 = id and 1 = k group by id, k").Rows())
}

func TestSetPlanCacheLimitSwitch(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = 1;")
tk.MustQuery("select @@session.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = off;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("0"))

tk.MustExec("set @@global.tidb_enable_plan_cache_for_param_limit = ON;")
tk.MustQuery("select @@global.tidb_enable_plan_cache_for_param_limit").Check(testkit.Rows("1"))
}

func TestPlanCacheLimitSwitchEffective(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, key(a))")

checkIfCached := func(res string) {
tk.MustExec("set @a = 1")
tk.MustExec("execute stmt using @a")
tk.MustExec("execute stmt using @a")
tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(res))
}

// before prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: query has 'limit ?' is un-cacheable"))
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")

// after prepare
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("execute stmt using @a")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 skip plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off"))
tk.MustExec("deallocate prepare stmt")

// after execute
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = ON")
tk.MustExec("prepare stmt from 'select * from t limit ?'")
checkIfCached("1")
tk.MustExec("set @@session.tidb_enable_plan_cache_for_param_limit = OFF")
checkIfCached("0")
tk.MustExec("deallocate prepare stmt")
}
2 changes: 1 addition & 1 deletion executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -645,7 +645,7 @@ func TestSetVar(t *testing.T) {
tk.MustQuery("select @@tidb_enable_tso_follower_proxy").Check(testkit.Rows("0"))
require.Error(t, tk.ExecToErr("set tidb_enable_tso_follower_proxy = 1"))

tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("0"))
tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("1"))
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("1"))
tk.MustExec("set global tidb_enable_historical_stats = 0")
Expand Down
6 changes: 4 additions & 2 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ import (
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/sem"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/slice"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stringutil"
"golang.org/x/exp/slices"
Expand Down Expand Up @@ -317,7 +316,10 @@ func (e *ShowExec) fetchShowBind() error {
} else {
tmp = domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord()
}
bindRecords := slice.Copy(tmp)
bindRecords := make([]*bindinfo.BindRecord, 0)
for _, bindRecord := range tmp {
bindRecords = append(bindRecords, bindRecord.Copy())
}
// Remove the invalid bindRecord.
ind := 0
for _, bindData := range bindRecords {
Expand Down
2 changes: 2 additions & 0 deletions infoschema/cluster_tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1053,6 +1053,8 @@ func TestSetBindingStatusBySQLDigest(t *testing.T) {
tk.MustExec(sql)
tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))
tk.MustGetErrMsg("set binding enabled for sql digest '2'", "can't find any binding for '2'")
tk.MustGetErrMsg("set binding enabled for sql digest ''", "sql digest is empty")
tk.MustGetErrMsg("set binding disabled for sql digest ''", "sql digest is empty")
}

func TestCreateBindingWhenCloseStmtSummaryTable(t *testing.T) {
Expand Down
102 changes: 102 additions & 0 deletions metrics/grafana/tidb.json
Original file line number Diff line number Diff line change
Expand Up @@ -18049,6 +18049,108 @@
"align": false,
"alignLevel": null
}
},
{
"aliasColors": {},
"bars": false,
"dashLength": 10,
"dashes": false,
"datasource": "${DS_TEST-CLUSTER}",
"description": "The TTL task statuses in each worker",
"fieldConfig": {
"defaults": {},
"overrides": []
},
"fill": 1,
"fillGradient": 0,
"gridPos": {
"h": 8,
"w": 12,
"x": 12,
"y": 100
},
"hiddenSeries": false,
"id": 294,
"legend": {
"avg": false,
"current": false,
"max": false,
"min": false,
"rightSide": true,
"show": true,
"total": false,
"values": false
},
"lines": true,
"linewidth": 1,
"nullPointMode": "null",
"options": {
"alertThreshold": true
},
"percentage": false,
"pluginVersion": "7.5.10",
"pointradius": 2,
"points": false,
"renderer": "flot",
"seriesOverrides": [
{
"alias": "running",
"color": "#5794F2"
}
],
"spaceLength": 10,
"stack": false,
"steppedLine": false,
"targets": [
{
"exemplar": true,
"expr": "sum(tidb_server_ttl_task_status{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type, instance)",
"interval": "",
"legendFormat": "{{ instance }} {{ type }}",
"queryType": "randomWalk",
"refId": "A"
}
],
"thresholds": [],
"timeFrom": null,
"timeRegions": [],
"timeShift": null,
"title": "TTL Task Count By Status",
"tooltip": {
"shared": true,
"sort": 0,
"value_type": "individual"
},
"type": "graph",
"xaxis": {
"buckets": null,
"mode": "time",
"name": null,
"show": true,
"values": []
},
"yaxes": [
{
"format": "short",
"label": null,
"logBase": 1,
"max": null,
"min": "0",
"show": true
},
{
"format": "short",
"label": null,
"logBase": 1,
"max": null,
"min": null,
"show": true
}
],
"yaxis": {
"align": false,
"alignLevel": null
}
}
],
"title": "TTL",
Expand Down
Loading

0 comments on commit 0b27925

Please sign in to comment.