diff --git a/DEPS.bzl b/DEPS.bzl index 0796b92cc9c72..2e78e45f88b6e 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2818,8 +2818,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:/92S0s/TCoCmK2vv6WbkXNeqtLn90sHRJ5Vlx1Sigas=", - version = "v0.0.0-20220913025519-586cff113d10", + sum = "h1:ceg4xjEEXNgPsScTQ5dtidiltLF4h17Y/jUqfyLAy9E=", + version = "v0.0.0-20220929075948-06e08d5ed64c", ) go_repository( name = "com_github_pingcap_log", @@ -3422,15 +3422,15 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:TxDSQAmtGdE34BvOaYF35mRrAXePeZEq8quvuAwrKsI=", - version = "v2.0.1-0.20220923061703-33efe476e022", + sum = "h1:/13jzD/AR7v3dCLweFQ2JG8bihh3HLVIci2tbOHHGW0=", + version = "v2.0.1-0.20221012074856-6def8d7b90c4", ) go_repository( name = "com_github_tikv_pd_client", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/pd/client", - sum = "h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww=", - version = "v0.0.0-20220725055910-7187a7ab72db", + sum = "h1:REQOR1XraH1fT9BCoNBPZs1CAe+w7VPLU+d+si7DLYo=", + version = "v0.0.0-20221010134149-d50e5fe43f14", ) go_repository( name = "com_github_timakin_bodyclose", @@ -4250,8 +4250,8 @@ def go_deps(): name = "org_golang_x_crypto", build_file_proto_mode = "disable_global", importpath = "golang.org/x/crypto", - sum = "h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg=", - version = "v0.0.0-20210921155107-089bfa567519", + sum = "h1:kUhD7nTDoI3fVd9G4ORWrbV5NY0liEs/Jg2pv5f+bBA=", + version = "v0.0.0-20220411220226-7b82a4e95df4", ) go_repository( name = "org_golang_x_exp", diff --git a/Makefile.common b/Makefile.common index 8c0788f1cc012..0df4b8e0ff289 100644 --- a/Makefile.common +++ b/Makefile.common @@ -116,5 +116,5 @@ TEST_COVERAGE_DIR := "test_coverage" ifneq ("$(CI)", "0") BAZEL_GLOBAL_CONFIG := --output_user_root=/home/jenkins/.tidb/tmp - BAZEL_CMD_CONFIG := --config=ci + BAZEL_CMD_CONFIG := --config=ci --repository_cache=/home/jenkins/.tidb/tmp endif diff --git a/bindinfo/BUILD.bazel b/bindinfo/BUILD.bazel index faffa0420b0b7..93b55bfab21c2 100644 --- a/bindinfo/BUILD.bazel +++ b/bindinfo/BUILD.bazel @@ -57,6 +57,7 @@ go_test( ], embed = [":bindinfo"], flaky = True, + race = "on", shard_count = 50, deps = [ "//config", diff --git a/br/pkg/lightning/common/util.go b/br/pkg/lightning/common/util.go index 4d9465eae1868..cc03f0ec68dca 100644 --- a/br/pkg/lightning/common/util.go +++ b/br/pkg/lightning/common/util.go @@ -295,7 +295,7 @@ func InterpolateMySQLString(s string) string { } // TableExists return whether table with specified name exists in target db -func TableExists(ctx context.Context, db *sql.DB, schema, table string) (bool, error) { +func TableExists(ctx context.Context, db utils.QueryExecutor, schema, table string) (bool, error) { query := "SELECT 1 from INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ?" var exist string err := db.QueryRowContext(ctx, query, schema, table).Scan(&exist) @@ -309,6 +309,21 @@ func TableExists(ctx context.Context, db *sql.DB, schema, table string) (bool, e } } +// SchemaExists return whether schema with specified name exists. +func SchemaExists(ctx context.Context, db utils.QueryExecutor, schema string) (bool, error) { + query := "SELECT 1 from INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME = ?" + var exist string + err := db.QueryRowContext(ctx, query, schema).Scan(&exist) + switch { + case err == nil: + return true, nil + case err == sql.ErrNoRows: + return false, nil + default: + return false, errors.Annotatef(err, "check schema exists failed") + } +} + // GetJSON fetches a page and parses it as JSON. The parsed result will be // stored into the `v`. The variable `v` must be a pointer to a type that can be // unmarshalled from JSON. diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 6f94a43d7d073..ba8faac2996a3 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -517,9 +517,15 @@ type restoreSchemaWorker struct { func (worker *restoreSchemaWorker) addJob(sqlStr string, job *schemaJob) error { stmts, err := createIfNotExistsStmt(worker.glue.GetParser(), sqlStr, job.dbName, job.tblName) if err != nil { - return err + worker.logger.Warn("failed to rewrite statement, will use raw input instead", + zap.String("db", job.dbName), + zap.String("table", job.tblName), + zap.String("statement", sqlStr), + zap.Error(err)) + job.stmts = []string{sqlStr} + } else { + job.stmts = stmts } - job.stmts = stmts return worker.appendJob(job) } @@ -656,7 +662,25 @@ loop: for _, stmt := range job.stmts { task := logger.Begin(zap.DebugLevel, fmt.Sprintf("execute SQL: %s", stmt)) err = sqlWithRetry.Exec(worker.ctx, "run create schema job", stmt) + if err != nil { + // try to imitate IF NOT EXISTS behavior for parsing errors + exists := false + switch job.stmtType { + case schemaCreateDatabase: + var err2 error + exists, err2 = common.SchemaExists(worker.ctx, session, job.dbName) + if err2 != nil { + task.Error("failed to check database existence", zap.Error(err2)) + } + case schemaCreateTable: + exists, _ = common.TableExists(worker.ctx, session, job.dbName, job.tblName) + } + if exists { + err = nil + } + } task.End(zap.ErrorLevel, err) + if err != nil { err = common.ErrCreateSchema.Wrap(err).GenWithStackByArgs(common.UniqueTable(job.dbName, job.tblName), job.stmtType.String()) worker.wg.Done() diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index e06e39e5925bd..d470d9d5a2655 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -143,6 +143,7 @@ go_test( "//types", "//util/codec", "//util/mathutil", + "@com_github_fsouza_fake_gcs_server//fakestorage", "@com_github_golang_protobuf//proto", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", diff --git a/br/tests/lightning_character_sets/greek.toml b/br/tests/lightning_character_sets/greek.toml new file mode 100644 index 0000000000000..261c42def21e0 --- /dev/null +++ b/br/tests/lightning_character_sets/greek.toml @@ -0,0 +1,2 @@ +[tikv-importer] +backend = "local" diff --git a/br/tests/lightning_character_sets/greek/charsets-schema-create.sql b/br/tests/lightning_character_sets/greek/charsets-schema-create.sql new file mode 100644 index 0000000000000..175084e6b9913 --- /dev/null +++ b/br/tests/lightning_character_sets/greek/charsets-schema-create.sql @@ -0,0 +1,2 @@ +/*!40101 SET NAMES binary*/; +CREATE DATABASE `charsets` /*!40100 DEFAULT CHARACTER SET greek */; diff --git a/br/tests/lightning_character_sets/greek/charsets.greek-schema.sql b/br/tests/lightning_character_sets/greek/charsets.greek-schema.sql new file mode 100644 index 0000000000000..c064a5af82d5e --- /dev/null +++ b/br/tests/lightning_character_sets/greek/charsets.greek-schema.sql @@ -0,0 +1,5 @@ +/*!40101 SET NAMES binary*/; +CREATE TABLE `greek` ( + `c` varchar(20) DEFAULT NULL, + PRIMARY KEY (`c`) +) ENGINE=InnoDB DEFAULT CHARSET=greek; diff --git a/br/tests/lightning_character_sets/greek/charsets.greek.sql b/br/tests/lightning_character_sets/greek/charsets.greek.sql new file mode 100644 index 0000000000000..72cc74e29af25 --- /dev/null +++ b/br/tests/lightning_character_sets/greek/charsets.greek.sql @@ -0,0 +1,3 @@ +/*!40101 SET NAMES binary*/; +INSERT INTO `greek` VALUES +('α'); diff --git a/br/tests/lightning_character_sets/run.sh b/br/tests/lightning_character_sets/run.sh index 3d62ed26360ca..d1a7ea5728d16 100755 --- a/br/tests/lightning_character_sets/run.sh +++ b/br/tests/lightning_character_sets/run.sh @@ -75,3 +75,17 @@ run_lightning --config "tests/$TEST_NAME/binary.toml" -d "tests/$TEST_NAME/mixed run_sql 'SELECT sum(`唯一键`) AS s FROM charsets.mixed' check_contains 's: 5291' +# test about unsupported charset in UTF-8 encoding dump files +# test local backend +run_lightning --config "tests/$TEST_NAME/greek.toml" -d "tests/$TEST_NAME/greek" 2>&1 | grep -q "Unknown character set: 'greek'" +run_sql 'DROP DATABASE IF EXISTS charsets;' +run_sql 'CREATE DATABASE charsets;' +run_sql 'CREATE TABLE charsets.greek (c VARCHAR(20) PRIMARY KEY);' +run_lightning --config "tests/$TEST_NAME/greek.toml" -d "tests/$TEST_NAME/greek" +run_sql "SELECT count(*) FROM charsets.greek WHERE c = 'α';" +check_contains 'count(*): 1' +# test tidb backend +run_sql 'TRUNCATE TABLE charsets.greek;' +run_lightning --config "tests/$TEST_NAME/greek.toml" -d "tests/$TEST_NAME/greek" --backend tidb +run_sql "SELECT count(*) FROM charsets.greek WHERE c = 'α';" +check_contains 'count(*): 1' diff --git a/config/config.go b/config/config.go index 6855fcc98b3ea..78fa8417a72bc 100644 --- a/config/config.go +++ b/config/config.go @@ -485,9 +485,9 @@ type Instance struct { ForcePriority string `toml:"tidb_force_priority" json:"tidb_force_priority"` MemoryUsageAlarmRatio float64 `toml:"tidb_memory_usage_alarm_ratio" json:"tidb_memory_usage_alarm_ratio"` // EnableCollectExecutionInfo enables the TiDB to collect execution info. - EnableCollectExecutionInfo bool `toml:"tidb_enable_collect_execution_info" json:"tidb_enable_collect_execution_info"` - PluginDir string `toml:"plugin_dir" json:"plugin_dir"` - PluginLoad string `toml:"plugin_load" json:"plugin_load"` + EnableCollectExecutionInfo AtomicBool `toml:"tidb_enable_collect_execution_info" json:"tidb_enable_collect_execution_info"` + PluginDir string `toml:"plugin_dir" json:"plugin_dir"` + PluginLoad string `toml:"plugin_load" json:"plugin_load"` // MaxConnections is the maximum permitted number of simultaneous client connections. MaxConnections uint32 `toml:"max_connections" json:"max_connections"` TiDBEnableDDL AtomicBool `toml:"tidb_enable_ddl" json:"tidb_enable_ddl"` @@ -864,7 +864,7 @@ var defaultConf = Config{ CheckMb4ValueInUTF8: *NewAtomicBool(true), ForcePriority: "NO_PRIORITY", MemoryUsageAlarmRatio: DefMemoryUsageAlarmRatio, - EnableCollectExecutionInfo: true, + EnableCollectExecutionInfo: *NewAtomicBool(true), PluginDir: "/data/deploy/plugin", PluginLoad: "", MaxConnections: 0, diff --git a/ddl/ddl_tiflash_api.go b/ddl/ddl_tiflash_api.go index 47c05272113eb..6c818be465de9 100644 --- a/ddl/ddl_tiflash_api.go +++ b/ddl/ddl_tiflash_api.go @@ -398,6 +398,10 @@ func getTiFlashTableSyncProgress(pollTiFlashContext *TiFlashManagementContext, t zap.Int64("tableID", tableID), zap.Int("tiflashPeerCount", tiflashPeerCount), zap.Int("regionCount", regionCount), zap.Uint64("replicaCount", replicaCount)) progress = 1 } + if progress < 1 { + logutil.BgLogger().Debug("TiFlash replica progress < 1.", + zap.Int64("tableID", tableID), zap.Int("tiflashPeerCount", tiflashPeerCount), zap.Int("regionCount", regionCount), zap.Uint64("replicaCount", replicaCount)) + } return types.TruncateFloatToString(progress, 2), nil } diff --git a/distsql/BUILD.bazel b/distsql/BUILD.bazel index 8934e398a5f0b..5839f55fbc52c 100644 --- a/distsql/BUILD.bazel +++ b/distsql/BUILD.bazel @@ -63,6 +63,7 @@ go_test( ], embed = [":distsql"], flaky = True, + race = "on", deps = [ "//kv", "//parser/charset", diff --git a/distsql/distsql.go b/distsql/distsql.go index 7db99424dc91f..19c2d6e3edea4 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -91,7 +91,7 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie SessionMemTracker: sctx.GetSessionVars().StmtCtx.MemTracker, EnabledRateLimitAction: enabledRateLimitAction, EventCb: eventCb, - EnableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo, + EnableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load(), } if kvReq.StoreType == kv.TiFlash { diff --git a/domain/BUILD.bazel b/domain/BUILD.bazel index b32697ed79534..1c4a433c79e86 100644 --- a/domain/BUILD.bazel +++ b/domain/BUILD.bazel @@ -52,6 +52,7 @@ go_library( "//util/execdetails", "//util/expensivequery", "//util/logutil", + "//util/memoryusagealarm", "//util/servermemorylimit", "//util/sqlexec", "@com_github_ngaut_pools//:pools", diff --git a/executor/executor.go b/executor/executor.go index 5e13783a489a3..d14f8a55a7de3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -2109,7 +2109,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { } else if vars.StmtCtx.InSelectStmt { sc.PrevAffectedRows = -1 } - if globalConfig.Instance.EnableCollectExecutionInfo { + if globalConfig.Instance.EnableCollectExecutionInfo.Load() { // In ExplainFor case, RuntimeStatsColl should not be reset for reuse, // because ExplainFor need to display the last statement information. reuseObj := vars.StmtCtx.RuntimeStatsColl diff --git a/executor/executor_issue_test.go b/executor/executor_issue_test.go index b5fb06dbdb94b..f0b52422c19a2 100644 --- a/executor/executor_issue_test.go +++ b/executor/executor_issue_test.go @@ -490,7 +490,7 @@ func TestIndexJoin31494(t *testing.T) { func TestFix31038(t *testing.T) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { - conf.Instance.EnableCollectExecutionInfo = false + conf.Instance.EnableCollectExecutionInfo.Store(false) }) store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/executor/executor_test.go b/executor/executor_test.go index feac2f5a7c48a..5063362462cb5 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5198,12 +5198,9 @@ func TestHistoryRead(t *testing.T) { require.Greater(t, snapshotTS, curVer1.Ver) require.Less(t, snapshotTS, curVer2.Ver) tk.MustQuery("select * from history_read").Check(testkit.Rows("1")) - _, err = tk.Exec("insert history_read values (2)") - require.Error(t, err) - _, err = tk.Exec("update history_read set a = 3 where a = 1") - require.Error(t, err) - _, err = tk.Exec("delete from history_read where a = 1") - require.Error(t, err) + tk.MustExecToErr("insert history_read values (2)") + tk.MustExecToErr("update history_read set a = 3 where a = 1") + tk.MustExecToErr("delete from history_read where a = 1") tk.MustExec("set @@tidb_snapshot = ''") tk.MustQuery("select * from history_read").Check(testkit.Rows("1", "2")) tk.MustExec("insert history_read values (3)") diff --git a/executor/explain.go b/executor/explain.go index 1fef25865bc59..7699751600b89 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -188,6 +188,17 @@ func (h *memoryDebugModeHandler) genInfo(status string, needProfile bool, heapIn return h.infoField, err } +func updateTriggerIntervalByHeapInUse(heapInUse uint64) (time.Duration, int) { + const GB uint64 = 1 << 30 + if heapInUse < 30*GB { + return 5 * time.Second, 6 + } else if heapInUse < 40*GB { + return 15 * time.Second, 2 + } else { + return 30 * time.Second, 1 + } +} + func (h *memoryDebugModeHandler) run() { var err error var fields []zap.Field @@ -213,7 +224,9 @@ func (h *memoryDebugModeHandler) run() { zap.String("minHeapInUse", memory.FormatBytes(h.minHeapInUse)), zap.Int64("alarmRatio", h.alarmRatio), ) - ticker, loop := time.NewTicker(5*time.Second), 0 + triggerInterval := 5 * time.Second + printMod := 6 + ticker, loop := time.NewTicker(triggerInterval), 0 for { select { case <-h.ctx.Done(): @@ -221,13 +234,15 @@ func (h *memoryDebugModeHandler) run() { case <-ticker.C: heapInUse, trackedMem := h.fetchCurrentMemoryUsage(h.autoGC) loop++ - if loop%6 == 0 { + if loop%printMod == 0 { fields, err = h.genInfo("running", false, int64(heapInUse), int64(trackedMem)) logutil.BgLogger().Info("Memory Debug Mode", fields...) if err != nil { return } } + triggerInterval, printMod = updateTriggerIntervalByHeapInUse(heapInUse) + ticker.Reset(triggerInterval) if !h.autoGC { if heapInUse > uint64(h.minHeapInUse) && trackedMem/100*uint64(100+h.alarmRatio) < heapInUse { diff --git a/executor/index_advise_test.go b/executor/index_advise_test.go index a9bdb36232424..3415ffe83537b 100644 --- a/executor/index_advise_test.go +++ b/executor/index_advise_test.go @@ -29,12 +29,9 @@ func TestIndexAdvise(t *testing.T) { tk := testkit.NewTestKit(t, store) - _, err := tk.Exec("index advise infile '/tmp/nonexistence.sql'") - require.EqualError(t, err, "Index Advise: don't support load file without local field") - _, err = tk.Exec("index advise local infile ''") - require.EqualError(t, err, "Index Advise: infile path is empty") - _, err = tk.Exec("index advise local infile '/tmp/nonexistence.sql' lines terminated by ''") - require.EqualError(t, err, "Index Advise: don't support advise index for SQL terminated by nil") + tk.MustGetErrMsg("index advise infile '/tmp/nonexistence.sql'", "Index Advise: don't support load file without local field") + tk.MustGetErrMsg("index advise local infile ''", "Index Advise: infile path is empty") + tk.MustGetErrMsg("index advise local infile '/tmp/nonexistence.sql' lines terminated by ''", "Index Advise: don't support advise index for SQL terminated by nil") path := "/tmp/index_advise.sql" fp, err := os.Create(path) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 2965597dc71fb..66ddbc2271afc 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -2223,6 +2223,8 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s progress += progressMap[p.ID] } progress = progress / float64(len(pi.Definitions)) + progressString := types.TruncateFloatToString(progress, 2) + progress, _ = strconv.ParseFloat(progressString, 64) } else { progress = progressMap[tbl.ID] } diff --git a/go.mod b/go.mod index 199183e997f6f..853421bf23d5d 100644 --- a/go.mod +++ b/go.mod @@ -67,7 +67,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10 + github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c github.com/pingcap/log v1.1.0 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e @@ -85,8 +85,8 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022 - github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db + github.com/tikv/client-go/v2 v2.0.1-0.20221012074856-6def8d7b90c4 + github.com/tikv/pd/client v0.0.0-20221010134149-d50e5fe43f14 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible @@ -226,7 +226,7 @@ require ( go.opentelemetry.io/otel/sdk/metric v0.20.0 // indirect go.opentelemetry.io/otel/trace v0.20.0 // indirect go.opentelemetry.io/proto/otlp v0.7.0 // indirect - golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 // indirect + golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4 // indirect golang.org/x/exp/typeparams v0.0.0-20220613132600-b0d781184e0d // indirect golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect golang.org/x/xerrors v0.0.0-20220411194840-2f41105eb62f // indirect diff --git a/go.sum b/go.sum index 88a44ead4a9fd..3b5ae428f06fd 100644 --- a/go.sum +++ b/go.sum @@ -755,9 +755,9 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20220510035547-0e2f26c0a46a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10 h1:/92S0s/TCoCmK2vv6WbkXNeqtLn90sHRJ5Vlx1Sigas= -github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220818063303-5c20f55db5ad/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c h1:ceg4xjEEXNgPsScTQ5dtidiltLF4h17Y/jUqfyLAy9E= +github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -905,10 +905,10 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022 h1:TxDSQAmtGdE34BvOaYF35mRrAXePeZEq8quvuAwrKsI= -github.com/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022/go.mod h1:6pedLz7wiINLHXwCT1+yMZmzuG42+ubtBkkfcwoukIo= -github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww= -github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db/go.mod h1:ew8kS0yIcEaSetuuywkTLIUBR+sz3J5XvAYRae11qwc= +github.com/tikv/client-go/v2 v2.0.1-0.20221012074856-6def8d7b90c4 h1:/13jzD/AR7v3dCLweFQ2JG8bihh3HLVIci2tbOHHGW0= +github.com/tikv/client-go/v2 v2.0.1-0.20221012074856-6def8d7b90c4/go.mod h1:gdXot2ofS2EOGtrXQ2qyESonQX/gFmgtfBCqCOSWg9E= +github.com/tikv/pd/client v0.0.0-20221010134149-d50e5fe43f14 h1:REQOR1XraH1fT9BCoNBPZs1CAe+w7VPLU+d+si7DLYo= +github.com/tikv/pd/client v0.0.0-20221010134149-d50e5fe43f14/go.mod h1:E/7+Fkqzwsrp4duzJ2gLPqFl6awU7QG+5yFRXaQwimM= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144/go.mod h1:Qimiffbc6q9tBWlVV6x0P9sat/ao1xEkREYPPj9hphk= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= @@ -1051,8 +1051,9 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= -golang.org/x/crypto v0.0.0-20210921155107-089bfa567519 h1:7I4JAnoQBe7ZtJcBaYHi5UtiO8tQHbUSXxL+pnGRANg= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4 h1:kUhD7nTDoI3fVd9G4ORWrbV5NY0liEs/Jg2pv5f+bBA= +golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index b8face4379eba..5e323b2adc88a 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -602,7 +602,7 @@ func TestStmtSummaryResultRows(t *testing.T) { tk.MustExec("set global tidb_stmt_summary_max_sql_length=4096") tk.MustExec("set global tidb_enable_stmt_summary=0") tk.MustExec("set global tidb_enable_stmt_summary=1") - if !config.GetGlobalConfig().Instance.EnableCollectExecutionInfo { + if !config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load() { tk.MustExec("set @@tidb_enable_collect_execution_info=1") defer tk.MustExec("set @@tidb_enable_collect_execution_info=0") } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 0e315c78186fc..15cb2026ba737 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1533,7 +1533,7 @@ func TestVariablesInfo(t *testing.T) { // stabalize timestamp val and EnableCollectExecutionInfo tk.MustExec("SET TIMESTAMP=123456789") - config.GetGlobalConfig().Instance.EnableCollectExecutionInfo = false + config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Store(false) // Test that in the current_value matches the default value in all // but a few permitted special cases. // See session/bootstrap.go:doDMLWorks() for where the exceptions are defined. diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 0c0090e353f77..0bf00b7c151b6 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -259,7 +259,9 @@ type IndexPartSpecification struct { Column *ColumnName Length int - Expr ExprNode + // Order is parsed but should be ignored because MySQL v5.7 doesn't support it. + Desc bool + Expr ExprNode } // Restore implements Node interface. @@ -270,6 +272,9 @@ func (n *IndexPartSpecification) Restore(ctx *format.RestoreCtx) error { return errors.Annotate(err, "An error occurred while splicing IndexPartSpecifications") } ctx.WritePlain(")") + if n.Desc { + ctx.WritePlain(" DESC") + } return nil } if err := n.Column.Restore(ctx); err != nil { @@ -278,6 +283,9 @@ func (n *IndexPartSpecification) Restore(ctx *format.RestoreCtx) error { if n.Length > 0 { ctx.WritePlainf("(%d)", n.Length) } + if n.Desc { + ctx.WritePlain(" DESC") + } return nil } diff --git a/parser/parser.go b/parser/parser.go index c330c3351222a..6eafe157e0a77 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -13458,12 +13458,11 @@ yynewstate: } case 330: { - // Order is parsed but just ignored as MySQL did. - parser.yyVAL.item = &ast.IndexPartSpecification{Column: yyS[yypt-2].item.(*ast.ColumnName), Length: yyS[yypt-1].item.(int)} + parser.yyVAL.item = &ast.IndexPartSpecification{Column: yyS[yypt-2].item.(*ast.ColumnName), Length: yyS[yypt-1].item.(int), Desc: yyS[yypt-0].item.(bool)} } case 331: { - parser.yyVAL.item = &ast.IndexPartSpecification{Expr: yyS[yypt-2].expr} + parser.yyVAL.item = &ast.IndexPartSpecification{Expr: yyS[yypt-2].expr, Desc: yyS[yypt-0].item.(bool)} } case 332: { diff --git a/parser/parser.y b/parser/parser.y index 417e252faab3e..b988926d4ebee 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -3701,12 +3701,11 @@ IndexPartSpecificationList: IndexPartSpecification: ColumnName OptFieldLen OptOrder { - // Order is parsed but just ignored as MySQL did. - $$ = &ast.IndexPartSpecification{Column: $1.(*ast.ColumnName), Length: $2.(int)} + $$ = &ast.IndexPartSpecification{Column: $1.(*ast.ColumnName), Length: $2.(int), Desc: $3.(bool)} } | '(' Expression ')' OptOrder { - $$ = &ast.IndexPartSpecification{Expr: $2} + $$ = &ast.IndexPartSpecification{Expr: $2, Desc: $4.(bool)} } IndexLockAndAlgorithmOpt: diff --git a/parser/parser_test.go b/parser/parser_test.go index db8286dba834b..21147a8040c45 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2364,6 +2364,8 @@ func TestDDL(t *testing.T) { {"CREATE TABLE foo (a.b, b);", false, ""}, {"CREATE TABLE foo (a, b.c);", false, ""}, {"CREATE TABLE (name CHAR(50) BINARY)", false, ""}, + {"CREATE TABLE foo (name CHAR(50) COLLATE ascii_bin PRIMARY KEY COLLATE latin1_bin, INDEX (name ASC))", true, "CREATE TABLE `foo` (`name` CHAR(50) COLLATE ascii_bin PRIMARY KEY COLLATE latin1_bin,INDEX(`name`))"}, + {"CREATE TABLE foo (name CHAR(50) COLLATE ascii_bin PRIMARY KEY COLLATE latin1_bin, INDEX (name DESC))", true, "CREATE TABLE `foo` (`name` CHAR(50) COLLATE ascii_bin PRIMARY KEY COLLATE latin1_bin,INDEX(`name` DESC))"}, // test enable or disable cached table {"ALTER TABLE tmp CACHE", true, "ALTER TABLE `tmp` CACHE"}, {"ALTER TABLE tmp NOCACHE", true, "ALTER TABLE `tmp` NOCACHE"}, @@ -2966,7 +2968,7 @@ func TestDDL(t *testing.T) { {"ALTER TABLE t ADD INDEX (a) USING RTREE COMMENT 'a'", true, "ALTER TABLE `t` ADD INDEX(`a`) USING RTREE COMMENT 'a'"}, {"ALTER TABLE t ADD KEY (a) USING HASH COMMENT 'a'", true, "ALTER TABLE `t` ADD INDEX(`a`) USING HASH COMMENT 'a'"}, {"ALTER TABLE t ADD KEY IF NOT EXISTS (a) USING HASH COMMENT 'a'", true, "ALTER TABLE `t` ADD INDEX IF NOT EXISTS(`a`) USING HASH COMMENT 'a'"}, - {"ALTER TABLE t ADD PRIMARY KEY ident USING RTREE ( a DESC , b )", true, "ALTER TABLE `t` ADD PRIMARY KEY `ident`(`a`, `b`) USING RTREE"}, + {"ALTER TABLE t ADD PRIMARY KEY ident USING RTREE ( a DESC , b )", true, "ALTER TABLE `t` ADD PRIMARY KEY `ident`(`a` DESC, `b`) USING RTREE"}, {"ALTER TABLE t ADD KEY USING RTREE ( a ) ", true, "ALTER TABLE `t` ADD INDEX(`a`) USING RTREE"}, {"ALTER TABLE t ADD KEY USING RTREE ( ident ASC , ident ( 123 ) )", true, "ALTER TABLE `t` ADD INDEX(`ident`, `ident`(123)) USING RTREE"}, {"ALTER TABLE t ADD PRIMARY KEY (a) COMMENT 'a'", true, "ALTER TABLE `t` ADD PRIMARY KEY(`a`) COMMENT 'a'"}, @@ -3439,7 +3441,7 @@ func TestDDL(t *testing.T) { {"create table a(a int, b int, key((a+1), (b+1)));", true, "CREATE TABLE `a` (`a` INT,`b` INT,INDEX((`a`+1), (`b`+1)))"}, {"create table a(a int, b int, key(a, (b+1)));", true, "CREATE TABLE `a` (`a` INT,`b` INT,INDEX(`a`, (`b`+1)))"}, {"create table a(a int, b int, key((a+1), b));", true, "CREATE TABLE `a` (`a` INT,`b` INT,INDEX((`a`+1), `b`))"}, - {"create table a(a int, b int, key((a + 1) desc));", true, "CREATE TABLE `a` (`a` INT,`b` INT,INDEX((`a`+1)))"}, + {"create table a(a int, b int, key((a + 1) desc));", true, "CREATE TABLE `a` (`a` INT,`b` INT,INDEX((`a`+1) DESC))"}, // for create sequence {"create sequence sequence", true, "CREATE SEQUENCE `sequence`"}, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b5a3d83ce6fd8..42690602b32cf 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -429,14 +429,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeInstance, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), Type: TypeBool, SetGlobal: func(s *SessionVars, val string) error { oldConfig := config.GetGlobalConfig() newValue := TiDBOptOn(val) - if oldConfig.Instance.EnableCollectExecutionInfo != newValue { + if oldConfig.Instance.EnableCollectExecutionInfo.Load() != newValue { newConfig := *oldConfig - newConfig.Instance.EnableCollectExecutionInfo = newValue + newConfig.Instance.EnableCollectExecutionInfo.Store(newValue) config.StoreGlobalConfig(&newConfig) } return nil }, GetGlobal: func(s *SessionVars) (string, error) { - return BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo), nil + return BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load()), nil }}, {Scope: ScopeInstance, Name: PluginLoad, Value: "", ReadOnly: true, GetGlobal: func(s *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.PluginLoad, nil diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index b6b25d0fe9fa3..a1b996d7468a8 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -343,7 +343,7 @@ func TestInstanceScopedVars(t *testing.T) { val, err = vars.GetSessionOrGlobalSystemVar(TiDBEnableCollectExecutionInfo) require.NoError(t, err) - require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo), val) + require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load()), val) val, err = vars.GetSessionOrGlobalSystemVar(TiDBConfig) require.NoError(t, err) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 0ccde1dda4a75..5c159bc355ab1 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -520,7 +520,7 @@ func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, startTs: startTs, vars: vars, needTriggerFallback: needTriggerFallback, - enableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo, + enableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load(), } go iter.run(ctxChild) return iter diff --git a/store/mockstore/unistore/BUILD.bazel b/store/mockstore/unistore/BUILD.bazel index cf2a6b23ed56d..50a04a77f9bfe 100644 --- a/store/mockstore/unistore/BUILD.bazel +++ b/store/mockstore/unistore/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/coprocessor", "@com_github_pingcap_kvproto//pkg/debugpb", "@com_github_pingcap_kvproto//pkg/errorpb", + "@com_github_pingcap_kvproto//pkg/keyspacepb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_kvproto//pkg/mpp", diff --git a/store/mockstore/unistore/pd.go b/store/mockstore/unistore/pd.go index 48f1b261bbed0..9361fcc9ddc07 100644 --- a/store/mockstore/unistore/pd.go +++ b/store/mockstore/unistore/pd.go @@ -20,6 +20,7 @@ import ( "math" "sync" + "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" us "github.com/pingcap/tidb/store/mockstore/unistore/tikv" pd "github.com/tikv/pd/client" @@ -163,3 +164,16 @@ func (c *pdClient) GetRegionFromMember(ctx context.Context, key []byte, memberUR func (c *pdClient) UpdateOption(option pd.DynamicOption, value interface{}) error { return nil } + +// LoadKeyspace loads and returns target keyspace's metadata. +func (c *pdClient) LoadKeyspace(ctx context.Context, name string) (*keyspacepb.KeyspaceMeta, error) { + return nil, nil +} + +// WatchKeyspaces watches keyspace meta changes. +// It returns a stream of slices of keyspace metadata. +// The first message in stream contains all current keyspaceMeta, +// all subsequent messages contains new put events for all keyspaces. +func (c *pdClient) WatchKeyspaces(ctx context.Context) (chan []*keyspacepb.KeyspaceMeta, error) { + return nil, nil +} diff --git a/tidb-server/main.go b/tidb-server/main.go index e42456617f9b2..13d7e59f8e3d1 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -568,7 +568,7 @@ func setGlobalVars() { case "check-mb4-value-in-utf8": cfg.Instance.CheckMb4ValueInUTF8.Store(cfg.CheckMb4ValueInUTF8.Load()) case "enable-collect-execution-info": - cfg.Instance.EnableCollectExecutionInfo = cfg.EnableCollectExecutionInfo + cfg.Instance.EnableCollectExecutionInfo.Store(cfg.EnableCollectExecutionInfo) case "max-server-connections": cfg.Instance.MaxConnections = cfg.MaxServerConnections case "run-ddl": diff --git a/util/BUILD.bazel b/util/BUILD.bazel index 65349065e4be5..879b7e86df089 100644 --- a/util/BUILD.bazel +++ b/util/BUILD.bazel @@ -59,6 +59,7 @@ go_test( "processinfo_test.go", "security_test.go", "urls_test.go", + "util_test.go", "wait_group_wrapper_test.go", ], data = glob(["tls_test/**"]), diff --git a/util/expensivequery/BUILD.bazel b/util/expensivequery/BUILD.bazel index 789639dc53452..7ecedc97b25c7 100644 --- a/util/expensivequery/BUILD.bazel +++ b/util/expensivequery/BUILD.bazel @@ -2,22 +2,14 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "expensivequery", - srcs = [ - "expensivequery.go", - "memory_usage_alarm.go", - ], + srcs = ["expensivequery.go"], importpath = "github.com/pingcap/tidb/util/expensivequery", visibility = ["//visibility:public"], deps = [ - "//config", - "//parser", "//sessionctx/variable", "//util", - "//util/disk", "//util/logutil", - "//util/memory", "@com_github_pingcap_log//:log", - "@org_golang_x_exp//slices", "@org_uber_go_zap//:zap", "@org_uber_go_zap//zapcore", ], @@ -30,11 +22,7 @@ go_test( embed = [":expensivequery"], flaky = True, deps = [ - "//sessionctx/stmtctx", "//testkit/testsetup", - "//util", - "//util/memory", - "@com_github_stretchr_testify//assert", "@org_uber_go_goleak//:goleak", ], ) diff --git a/util/gctuner/BUILD.bazel b/util/gctuner/BUILD.bazel index 262a280204f3e..d20f8b6a5b418 100644 --- a/util/gctuner/BUILD.bazel +++ b/util/gctuner/BUILD.bazel @@ -5,12 +5,15 @@ go_library( srcs = [ "finalizer.go", "mem.go", + "memory_limit_tuner.go", "tuner.go", ], importpath = "github.com/pingcap/tidb/util/gctuner", visibility = ["//visibility:public"], deps = [ "//util", + "//util/memory", + "@com_github_pingcap_failpoint//:failpoint", "@org_uber_go_atomic//:atomic", ], ) @@ -20,12 +23,15 @@ go_test( srcs = [ "finalizer_test.go", "mem_test.go", + "memory_limit_tuner_test.go", "tuner_test.go", ], embed = [":gctuner"], flaky = True, + race = "on", deps = [ - "@com_github_stretchr_testify//assert", + "//util/memory", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", ], ) diff --git a/util/gctuner/memory_limit_tuner.go b/util/gctuner/memory_limit_tuner.go index 15d447a77cb61..7a9035a387b39 100644 --- a/util/gctuner/memory_limit_tuner.go +++ b/util/gctuner/memory_limit_tuner.go @@ -36,7 +36,7 @@ type memoryLimitTuner struct { isTuning atomicutil.Bool percentage atomicutil.Float64 waitingReset atomicutil.Bool - nextGCTriggeredByMemoryLimit bool + nextGCTriggeredByMemoryLimit atomicutil.Bool } // tuning check the memory nextGC and judge whether this GC is trigger by memory limit. @@ -59,7 +59,7 @@ func (t *memoryLimitTuner) tuning() { // - Only if NextGC >= MemoryLimit , the **next** GC will be triggered by MemoryLimit. Thus, we need to reset // MemoryLimit after the **next** GC happens if needed. if float64(r.HeapInuse)*ratio > float64(debug.SetMemoryLimit(-1)) { - if t.nextGCTriggeredByMemoryLimit && t.waitingReset.CompareAndSwap(false, true) { + if t.nextGCTriggeredByMemoryLimit.Load() && t.waitingReset.CompareAndSwap(false, true) { go func() { debug.SetMemoryLimit(math.MaxInt64) resetInterval := 1 * time.Minute // Wait 1 minute and set back, to avoid frequent GC @@ -75,9 +75,9 @@ func (t *memoryLimitTuner) tuning() { } }() } - t.nextGCTriggeredByMemoryLimit = true + t.nextGCTriggeredByMemoryLimit.Store(true) } else { - t.nextGCTriggeredByMemoryLimit = false + t.nextGCTriggeredByMemoryLimit.Store(false) } } diff --git a/util/gctuner/memory_limit_tuner_test.go b/util/gctuner/memory_limit_tuner_test.go index 416ba245cabdf..47d1d8409d8b5 100644 --- a/util/gctuner/memory_limit_tuner_test.go +++ b/util/gctuner/memory_limit_tuner_test.go @@ -63,7 +63,7 @@ func TestGlobalMemoryTuner(t *testing.T) { time.Sleep(1 * time.Second) // If test.count > 1, wait tuning finished. require.True(t, GlobalMemoryLimitTuner.isTuning.Load()) require.False(t, GlobalMemoryLimitTuner.waitingReset.Load()) - require.Equal(t, GlobalMemoryLimitTuner.nextGCTriggeredByMemoryLimit, false) + require.Equal(t, GlobalMemoryLimitTuner.nextGCTriggeredByMemoryLimit.Load(), false) }() allocator := &mockAllocator{} diff --git a/util/memoryusagealarm/BUILD.bazel b/util/memoryusagealarm/BUILD.bazel new file mode 100644 index 0000000000000..48c7910492ecc --- /dev/null +++ b/util/memoryusagealarm/BUILD.bazel @@ -0,0 +1,34 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "memoryusagealarm", + srcs = ["memoryusagealarm.go"], + importpath = "github.com/pingcap/tidb/util/memoryusagealarm", + visibility = ["//visibility:public"], + deps = [ + "//config", + "//sessionctx/variable", + "//util", + "//util/disk", + "//util/logutil", + "//util/memory", + "@org_golang_x_exp//slices", + "@org_uber_go_zap//:zap", + "@org_uber_go_zap//zapcore", + ], +) + +go_test( + name = "memoryusagealarm_test", + srcs = ["memoryusagealarm_test.go"], + embed = [":memoryusagealarm"], + flaky = True, + race = "on", + deps = [ + "//sessionctx/stmtctx", + "//util", + "//util/execdetails", + "//util/memory", + "@com_github_stretchr_testify//assert", + ], +)