Skip to content

Commit

Permalink
session, sessionctx: enable metadata lock by default (#38866)
Browse files Browse the repository at this point in the history
close #37275, ref #38890
  • Loading branch information
wjhuang2016 authored Nov 7, 2022
1 parent d323b8d commit 490e4c4
Show file tree
Hide file tree
Showing 18 changed files with 65 additions and 19 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -409,7 +409,7 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare
-- //... -//cmd/... -//tests/graceshutdown/... \
-//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/...
bazel $(BAZEL_GLOBAL_CONFIG) coverage $(BAZEL_CMD_CONFIG) \
--build_event_json_file=bazel_2.json --@io_bazel_rules_go//go/config:cover_format=go_cover --define gotags=featuretag \
--build_event_json_file=bazel_2.json --@io_bazel_rules_go//go/config:cover_format=go_cover \
-- //... -//cmd/... -//tests/graceshutdown/... \
-//tests/globalkilltest/... -//tests/readonlytest/... -//br/pkg/task:task_test -//tests/realtikvtest/...

Expand Down
2 changes: 1 addition & 1 deletion ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ go_library(

go_test(
name = "ddl_test",
timeout = "moderate",
timeout = "long",
srcs = [
"attributes_sql_test.go",
"callback_test.go",
Expand Down
2 changes: 1 addition & 1 deletion executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ go_library(

go_test(
name = "executor_test",
timeout = "moderate",
timeout = "long",
srcs = [
"adapter_test.go",
"admin_test.go",
Expand Down
2 changes: 1 addition & 1 deletion executor/seqtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "seqtest_test",
timeout = "moderate",
timeout = "long",
srcs = [
"main_test.go",
"prepared_test.go",
Expand Down
2 changes: 1 addition & 1 deletion executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -938,7 +938,7 @@ func TestBatchInsertDelete(t *testing.T) {
atomic.StoreUint64(&kv.TxnTotalSizeLimit, originLimit)
}()
// Set the limitation to a small value, make it easier to reach the limitation.
atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5700)
atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5800)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
Expand Down
1 change: 1 addition & 0 deletions planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -1727,6 +1727,7 @@ func (p *preprocessor) updateStateFromStaleReadProcessor() error {
p.LastSnapshotTS = p.staleReadProcessor.GetStalenessReadTS()
p.SnapshotTSEvaluator = p.staleReadProcessor.GetStalenessTSEvaluatorForPrepare()
p.InfoSchema = p.staleReadProcessor.GetStalenessInfoSchema()
p.InfoSchema = &infoschema.SessionExtendedInfoSchema{InfoSchema: p.InfoSchema}
// If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction
// or is affected by the tidb_read_staleness session variable, then the statement will be makred as isStaleness
// in stmtCtx
Expand Down
2 changes: 2 additions & 0 deletions planner/funcdep/extract_fd_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,7 @@ func TestFDSet_ExtractFD(t *testing.T) {

ctx := context.TODO()
is := testGetIS(t, tk.Session())
is = &infoschema.SessionExtendedInfoSchema{InfoSchema: is}
for i, tt := range tests {
comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql)
require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO()))
Expand Down Expand Up @@ -308,6 +309,7 @@ func TestFDSet_ExtractFDForApply(t *testing.T) {

ctx := context.TODO()
is := testGetIS(t, tk.Session())
is = &infoschema.SessionExtendedInfoSchema{InfoSchema: is}
for i, tt := range tests {
require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO()))
require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), nil))
Expand Down
2 changes: 1 addition & 1 deletion server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ go_library(

go_test(
name = "server_test",
timeout = "moderate",
timeout = "long",
srcs = [
"column_test.go",
"conn_stmt_test.go",
Expand Down
46 changes: 41 additions & 5 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -641,13 +641,14 @@ const (
version97 = 97
// version98 add a column `Token_issuer` to `mysql.user`
version98 = 98
// version99 converts server-memory-quota to a sysvar
version99 = 99
// version100 converts server-memory-quota to a sysvar
version100 = 100
)

// currentBootstrapVersion is defined as a variable, so we can modify its value for testing.
// please make sure this is the largest version
var currentBootstrapVersion int64 = version99
var currentBootstrapVersion int64 = version100

// DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it.
var internalSQLTimeout = owner.ManagerSessionTTL + 15
Expand Down Expand Up @@ -748,10 +749,10 @@ var (
upgradeToVer93,
upgradeToVer94,
upgradeToVer95,
// We will redo upgradeToVer96 in upgradeToVer99, it is skipped here.
// We will redo upgradeToVer96 in upgradeToVer100, it is skipped here.
upgradeToVer97,
upgradeToVer98,
upgradeToVer99,
upgradeToVer100,
}
)

Expand Down Expand Up @@ -832,9 +833,13 @@ func upgrade(s Session) {
}
}
// Do upgrade works then update bootstrap version.
needEnableMdl := upgradeToVer99Before(s, ver)
for _, upgrade := range bootstrapVersion {
upgrade(s, ver)
}
if needEnableMdl {
upgradeToVer99After(s, ver)
}

variable.DDLForce2Queue.Store(false)
updateBootstrapVer(s)
Expand Down Expand Up @@ -1982,10 +1987,41 @@ func upgradeToVer98(s Session, ver int64) {
doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN IF NOT EXISTS `Token_issuer` varchar(255)")
}

func upgradeToVer99(s Session, ver int64) {
func upgradeToVer99Before(s Session, ver int64) bool {
if ver >= version99 {
return false
}
// Check if tidb_enable_metadata_lock exists in mysql.GLOBAL_VARIABLES.
// If not, insert "tidb_enable_metadata_lock | 0" since concurrent DDL may not be enabled.
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap)
rs, err := s.ExecuteInternal(ctx, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?;",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL)
terror.MustNil(err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
terror.MustNil(err)
if req.NumRows() != 0 {
return false
}

mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 0)
return true
}

func upgradeToVer99After(s Session, ver int64) {
if ver >= version99 {
return
}
sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = %[4]d WHERE VARIABLE_NAME = '%[3]s'",
mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 1)
mustExecute(s, sql)
}

func upgradeToVer100(s Session, ver int64) {
if ver >= version100 {
return
}
valStr := strconv.Itoa(int(config.GetGlobalConfig().Performance.ServerMemoryQuota))
importConfigOption(s, "performance.server-memory-quota", variable.TiDBServerMemoryLimit, valStr)
}
Expand Down
2 changes: 1 addition & 1 deletion session/session_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "session_test_test",
timeout = "moderate",
timeout = "long",
srcs = [
"main_test.go",
"session_test.go",
Expand Down
4 changes: 2 additions & 2 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1059,7 +1059,7 @@ const (
DefEnableTiDBGCAwareMemoryTrack = true
DefTiDBDefaultStrMatchSelectivity = 0.8
DefTiDBEnableTmpStorageOnOOM = true
DefTiDBEnableMDL = false
DefTiDBEnableMDL = true
DefTiFlashFastScan = false
DefMemoryUsageAlarmRatio = 0.7
DefMemoryUsageAlarmKeepRecordNum = 5
Expand Down Expand Up @@ -1134,7 +1134,7 @@ var (
EnableConcurrentDDL = atomic.NewBool(DefTiDBEnableConcurrentDDL)
DDLForce2Queue = atomic.NewBool(false)
EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables)
EnableMDL = atomic.NewBool(DefTiDBEnableMDL)
EnableMDL = atomic.NewBool(false)
AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize)
// EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance.
EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg)
Expand Down
6 changes: 6 additions & 0 deletions sessiontxn/isolation/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,12 @@ func (p *baseTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema {
if is := p.sctx.GetSessionVars().SnapshotInfoschema; is != nil {
return is.(infoschema.InfoSchema)
}
if _, ok := p.infoSchema.(*infoschema.SessionExtendedInfoSchema); !ok {
p.infoSchema = &infoschema.SessionExtendedInfoSchema{
InfoSchema: p.infoSchema,
}
p.sctx.GetSessionVars().TxnCtx.InfoSchema = p.infoSchema
}
return p.infoSchema
}

Expand Down
2 changes: 1 addition & 1 deletion statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ go_library(

go_test(
name = "handle_test",
timeout = "moderate",
timeout = "long",
srcs = [
"ddl_test.go",
"dump_test.go",
Expand Down
1 change: 1 addition & 0 deletions telemetry/data_feature_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,7 @@ func TestAddIndexAccelerationAndMDL(t *testing.T) {

allow := ddl.IsEnableFastReorg()
require.Equal(t, false, allow)
tk.MustExec("set global tidb_enable_metadata_lock = 0")
tk.MustExec("use test")
tk.MustExec("drop table if exists tele_t")
tk.MustExec("create table tele_t(id int, b int)")
Expand Down
2 changes: 1 addition & 1 deletion tests/realtikvtest/brietest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "brietest_test",
timeout = "moderate",
timeout = "long",
srcs = [
"backup_restore_test.go",
"binlog_test.go",
Expand Down
2 changes: 1 addition & 1 deletion tests/realtikvtest/sessiontest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "sessiontest_test",
timeout = "moderate",
timeout = "long",
srcs = [
"main_test.go",
"paging_test.go",
Expand Down
2 changes: 1 addition & 1 deletion tests/realtikvtest/statisticstest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "statisticstest_test",
timeout = "moderate",
timeout = "long",
srcs = [
"main_test.go",
"statistics_test.go",
Expand Down
2 changes: 1 addition & 1 deletion tests/realtikvtest/txntest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "txntest_test",
timeout = "moderate",
timeout = "long",
srcs = [
"isolation_test.go",
"main_test.go",
Expand Down

0 comments on commit 490e4c4

Please sign in to comment.