From 9e892c2691082667c1f8cb461b467111967bcf80 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 3 Nov 2023 23:47:09 +0800 Subject: [PATCH] This is an automated cherry-pick of #48264 Signed-off-by: ti-chi-bot --- ddl/metadatalocktest/BUILD.bazel | 4 + executor/analyze.go | 107 ++++++++- executor/analyze_utils.go | 4 +- executor/builder.go | 6 + .../memorycontrol/memory_control_test.go | 203 ++++++++++++++++++ 5 files changed, 314 insertions(+), 10 deletions(-) create mode 100644 pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go diff --git a/ddl/metadatalocktest/BUILD.bazel b/ddl/metadatalocktest/BUILD.bazel index d458d7d592368..8b4f84422e454 100644 --- a/ddl/metadatalocktest/BUILD.bazel +++ b/ddl/metadatalocktest/BUILD.bazel @@ -8,7 +8,11 @@ go_test( "mdl_test.go", ], flaky = True, +<<<<<<< HEAD:ddl/metadatalocktest/BUILD.bazel shard_count = 32, +======= + shard_count = 4, +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/test/analyzetest/memorycontrol/BUILD.bazel deps = [ "//config", "//ddl", diff --git a/executor/analyze.go b/executor/analyze.go index dd5c20440638e..4f7344de31cd5 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -55,6 +55,12 @@ type AnalyzeExec struct { wg util.WaitGroupWrapper opts map[ast.AnalyzeOptionType]uint64 OptionsMap map[int64]core.V2AnalyzeOptions +<<<<<<< HEAD:executor/analyze.go +======= + gp *gp.Pool + // errExitCh is used to notice the worker that the whole analyze task is finished when to meet error. + errExitCh chan struct{} +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/analyze.go } var ( @@ -165,6 +171,17 @@ func (e *AnalyzeExec) Next(ctx context.Context, _ *chunk.Chunk) error { for i := 0; i < concurrency; i++ { e.wg.Run(func() { e.analyzeWorker(taskCh, resultsCh) }) } +<<<<<<< HEAD:executor/analyze.go +======= + pruneMode := variable.PartitionPruneMode(sessionVars.PartitionPruneMode.Load()) + // needGlobalStats used to indicate whether we should merge the partition-level stats to global-level stats. + needGlobalStats := pruneMode == variable.Dynamic + globalStatsMap := make(map[globalStatsKey]globalStatsInfo) + g, _ := errgroup.WithContext(ctx) + g.Go(func() error { + return e.handleResultsError(ctx, concurrency, needGlobalStats, globalStatsMap, resultsCh, len(tasks)) + }) +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/analyze.go for _, task := range tasks { prepareV2AnalyzeJobInfo(task.colExec, false) AddNewAnalyzeJob(e.ctx, task.job) @@ -177,6 +194,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, _ *chunk.Chunk) error { taskCh <- task } close(taskCh) +<<<<<<< HEAD:executor/analyze.go e.wg.Wait() close(resultsCh) pruneMode := variable.PartitionPruneMode(e.ctx.GetSessionVars().PartitionPruneMode.Load()) @@ -187,11 +205,21 @@ func (e *AnalyzeExec) Next(ctx context.Context, _ *chunk.Chunk) error { for _, task := range tasks { if task.colExec != nil && task.colExec.memTracker != nil { task.colExec.memTracker.Detach() +======= + defer func() { + for _, task := range tasks { + if task.colExec != nil && task.colExec.memTracker != nil { + task.colExec.memTracker.Detach() + } +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/analyze.go } - } + }() + + err = e.waitFinish(ctx, g, resultsCh) if err != nil { return err } + failpoint.Inject("mockKillFinishedAnalyzeJob", func() { dom := domain.GetDomain(e.ctx) dom.SysProcTracker().KillSysProcess(util.GetAutoAnalyzeProcID(dom.ServerID)) @@ -206,8 +234,49 @@ func (e *AnalyzeExec) Next(ctx context.Context, _ *chunk.Chunk) error { if err != nil { e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) } +<<<<<<< HEAD:executor/analyze.go if e.ctx.GetSessionVars().InRestrictedSQL { return statsHandle.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema)) +======= + return statsHandle.Update(infoSchema) +} + +func (e *AnalyzeExec) waitFinish(ctx context.Context, g *errgroup.Group, resultsCh chan *statistics.AnalyzeResults) error { + checkwg, _ := errgroup.WithContext(ctx) + checkwg.Go(func() error { + // It is to wait for the completion of the result handler. if the result handler meets error, we should cancel + // the analyze process by closing the errExitCh. + err := g.Wait() + if err != nil { + close(e.errExitCh) + return err + } + return nil + }) + checkwg.Go(func() error { + // Wait all workers done and close the results channel. + e.wg.Wait() + close(resultsCh) + return nil + }) + return checkwg.Wait() +} + +// filterAndCollectTasks filters the tasks that are not locked and collects the table IDs. +func filterAndCollectTasks(tasks []*analyzeTask, statsHandle *handle.Handle, infoSchema infoschema.InfoSchema) ([]*analyzeTask, uint, []string, error) { + var ( + filteredTasks []*analyzeTask + skippedTables []string + needAnalyzeTableCnt uint + // tidMap is used to deduplicate table IDs. + // In stats v1, analyze for each index is a single task, and they have the same table id. + tidAndPidsMap = make(map[int64]struct{}, len(tasks)) + ) + + lockedTableAndPartitionIDs, err := getLockedTableAndPartitionIDs(statsHandle, tasks) + if err != nil { + return nil, 0, nil, err +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/analyze.go } return statsHandle.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema), handle.WithTableStatsByQuery()) } @@ -421,6 +490,7 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultsCh chan<- StartAnalyzeJob(e.ctx, task.job) switch task.taskType { case colTask: +<<<<<<< HEAD:executor/analyze.go resultsCh <- analyzeColumnsPushDownEntry(task.colExec) case idxTask: resultsCh <- analyzeIndexPushdown(task.idxExec) @@ -430,6 +500,19 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultsCh chan<- resultsCh <- analyzePKIncremental(task.colIncrementalExec) case idxIncrementalTask: resultsCh <- analyzeIndexIncremental(task.idxIncrementalExec) +======= + select { + case <-e.errExitCh: + return + case resultsCh <- analyzeColumnsPushDownEntry(e.gp, task.colExec): + } + case idxTask: + select { + case <-e.errExitCh: + return + case resultsCh <- analyzeIndexPushdown(task.idxExec): + } +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/analyze.go } } } @@ -584,16 +667,24 @@ func finishJobWithLog(sctx sessionctx.Context, job *statistics.AnalyzeJob, analy var state string if analyzeErr != nil { state = statistics.AnalyzeFailed + logutil.BgLogger().Warn(fmt.Sprintf("analyze table `%s`.`%s` has %s", job.DBName, job.TableName, state), + zap.String("partition", job.PartitionName), + zap.String("job info", job.JobInfo), + zap.Time("start time", job.StartTime), + zap.Time("end time", job.EndTime), + zap.String("cost", job.EndTime.Sub(job.StartTime).String()), + zap.String("sample rate reason", job.SampleRateReason), + zap.Error(analyzeErr)) } else { state = statistics.AnalyzeFinished + logutil.BgLogger().Info(fmt.Sprintf("analyze table `%s`.`%s` has %s", job.DBName, job.TableName, state), + zap.String("partition", job.PartitionName), + zap.String("job info", job.JobInfo), + zap.Time("start time", job.StartTime), + zap.Time("end time", job.EndTime), + zap.String("cost", job.EndTime.Sub(job.StartTime).String()), + zap.String("sample rate reason", job.SampleRateReason)) } - logutil.BgLogger().Info(fmt.Sprintf("analyze table `%s`.`%s` has %s", job.DBName, job.TableName, state), - zap.String("partition", job.PartitionName), - zap.String("job info", job.JobInfo), - zap.Time("start time", job.StartTime), - zap.Time("end time", job.EndTime), - zap.String("cost", job.EndTime.Sub(job.StartTime).String()), - zap.String("sample rate reason", job.SampleRateReason)) } } diff --git a/executor/analyze_utils.go b/executor/analyze_utils.go index e7397df158a22..26ce925668906 100644 --- a/executor/analyze_utils.go +++ b/executor/analyze_utils.go @@ -49,7 +49,7 @@ func isAnalyzeWorkerPanic(err error) bool { func getAnalyzePanicErr(r interface{}) error { if msg, ok := r.(string); ok { if msg == globalPanicAnalyzeMemoryExceed { - return errAnalyzeOOM + return errors.Trace(errAnalyzeOOM) } if strings.Contains(msg, memory.PanicMemoryExceedWarnMsg) { return errors.Errorf(msg, errAnalyzeOOM) @@ -61,7 +61,7 @@ func getAnalyzePanicErr(r interface{}) error { } return err } - return errAnalyzeWorkerPanic + return errors.Trace(errAnalyzeWorkerPanic) } // analyzeResultsNotifyWaitGroupWrapper is a wrapper for sync.WaitGroup diff --git a/executor/builder.go b/executor/builder.go index 4ed4988dd7c92..b7843701976a5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2998,6 +2998,12 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) Executor { tasks: make([]*analyzeTask, 0, len(v.ColTasks)+len(v.IdxTasks)), opts: v.Opts, OptionsMap: v.OptionsMap, +<<<<<<< HEAD:executor/builder.go +======= + wg: util.NewWaitGroupPool(gp), + gp: gp, + errExitCh: make(chan struct{}), +>>>>>>> 4f00ece106b (executor: fix hang for analyze table when exceed tidb_mem_quota_analyze (#48264)):pkg/executor/builder.go } enableFastAnalyze := b.ctx.GetSessionVars().EnableFastAnalyze autoAnalyze := "" diff --git a/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go b/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go new file mode 100644 index 0000000000000..b81c0de7efc49 --- /dev/null +++ b/pkg/executor/test/analyzetest/memorycontrol/memory_control_test.go @@ -0,0 +1,203 @@ +// Copyright 2023 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package memorycontrol + +import ( + "fmt" + "runtime" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/executor" + "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util" + "github.com/stretchr/testify/require" +) + +func TestGlobalMemoryControlForAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk0 := testkit.NewTestKit(t, store) + tk0.MustExec("set global tidb_mem_oom_action = 'cancel'") + tk0.MustExec("set global tidb_server_memory_limit = 512MB") + tk0.MustExec("set global tidb_server_memory_limit_sess_min_size = 128") + + sm := &testkit.MockSessionManager{ + PS: []*util.ProcessInfo{tk0.Session().ShowProcess()}, + } + dom.ServerMemoryLimitHandle().SetSessionManager(sm) + go dom.ServerMemoryLimitHandle().Run() + + tk0.MustExec("use test") + tk0.MustExec("create table t(a int)") + tk0.MustExec("insert into t select 1") + for i := 1; i <= 8; i++ { + tk0.MustExec("insert into t select * from t") // 256 Lines + } + sql := "analyze table t with 1.0 samplerate;" // Need about 100MB + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats", `return(536870912)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume", `return(100)`)) + _, err := tk0.Exec(sql) + require.True(t, strings.Contains(err.Error(), "Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory. Please try narrowing your query scope or increase the tidb_server_memory_limit and try again.")) + runtime.GC() + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume")) + tk0.MustExec(sql) +} + +func TestGlobalMemoryControlForPrepareAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk0 := testkit.NewTestKit(t, store) + tk0.MustExec("set global tidb_mem_oom_action = 'cancel'") + tk0.MustExec("set global tidb_mem_quota_query = 209715200 ") // 200MB + tk0.MustExec("set global tidb_server_memory_limit = 5GB") + tk0.MustExec("set global tidb_server_memory_limit_sess_min_size = 128") + + sm := &testkit.MockSessionManager{ + PS: []*util.ProcessInfo{tk0.Session().ShowProcess()}, + } + dom.ServerMemoryLimitHandle().SetSessionManager(sm) + go dom.ServerMemoryLimitHandle().Run() + + tk0.MustExec("use test") + tk0.MustExec("create table t(a int)") + tk0.MustExec("insert into t select 1") + for i := 1; i <= 8; i++ { + tk0.MustExec("insert into t select * from t") // 256 Lines + } + sqlPrepare := "prepare stmt from 'analyze table t with 1.0 samplerate';" + sqlExecute := "execute stmt;" // Need about 100MB + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats", `return(536870912)`)) // 512MB + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume", `return(100)`)) + runtime.GC() + // won't be killed by tidb_mem_quota_query + tk0.MustExec(sqlPrepare) + tk0.MustExec(sqlExecute) + runtime.GC() + // killed by tidb_server_memory_limit + tk0.MustExec("set global tidb_server_memory_limit = 512MB") + _, err0 := tk0.Exec(sqlPrepare) + require.NoError(t, err0) + _, err1 := tk0.Exec(sqlExecute) + // Killed and the WarnMsg is WarnMsgSuffixForInstance instead of WarnMsgSuffixForSingleQuery + require.True(t, strings.Contains(err1.Error(), "Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory. Please try narrowing your query scope or increase the tidb_server_memory_limit and try again.")) + runtime.GC() + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume")) + tk0.MustExec(sqlPrepare) + tk0.MustExec(sqlExecute) +} + +func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + originalVal1 := tk.MustQuery("select @@global.tidb_mem_oom_action").Rows()[0][0].(string) + tk.MustExec("set global tidb_mem_oom_action = 'cancel'") + //originalVal2 := tk.MustQuery("select @@global.tidb_server_memory_limit").Rows()[0][0].(string) + tk.MustExec("set global tidb_server_memory_limit = 512MB") + originalVal3 := tk.MustQuery("select @@global.tidb_server_memory_limit_sess_min_size").Rows()[0][0].(string) + tk.MustExec("set global tidb_server_memory_limit_sess_min_size = 128") + defer func() { + tk.MustExec(fmt.Sprintf("set global tidb_mem_oom_action = %v", originalVal1)) + //tk.MustExec(fmt.Sprintf("set global tidb_server_memory_limit = %v", originalVal2)) + tk.MustExec(fmt.Sprintf("set global tidb_server_memory_limit_sess_min_size = %v", originalVal3)) + }() + + // clean child trackers + oldChildTrackers := executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest() + for _, tracker := range oldChildTrackers { + tracker.Detach() + } + defer func() { + for _, tracker := range oldChildTrackers { + tracker.AttachTo(executor.GlobalAnalyzeMemoryTracker) + } + }() + childTrackers := executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest() + require.Len(t, childTrackers, 0) + + tk.MustExec("use test") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t select 1") + for i := 1; i <= 8; i++ { + tk.MustExec("insert into t select * from t") // 256 Lines + } + _, err0 := tk.Exec("analyze table t with 1.0 samplerate;") + require.NoError(t, err0) + rs0 := tk.MustQuery("select fail_reason from mysql.analyze_jobs where table_name=? and state=? limit 1", "t", "failed") + require.Len(t, rs0.Rows(), 0) + + h := dom.StatsHandle() + originalVal4 := autoanalyze.AutoAnalyzeMinCnt + originalVal5 := tk.MustQuery("select @@global.tidb_auto_analyze_ratio").Rows()[0][0].(string) + autoanalyze.AutoAnalyzeMinCnt = 0 + tk.MustExec("set global tidb_auto_analyze_ratio = 0.001") + defer func() { + autoanalyze.AutoAnalyzeMinCnt = originalVal4 + tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_ratio = %v", originalVal5)) + }() + + sm := &testkit.MockSessionManager{ + Dom: dom, + PS: []*util.ProcessInfo{tk.Session().ShowProcess()}, + } + dom.ServerMemoryLimitHandle().SetSessionManager(sm) + go dom.ServerMemoryLimitHandle().Run() + + tk.MustExec("insert into t values(4),(5),(6)") + require.NoError(t, h.DumpStatsDeltaToKV(true)) + err := h.Update(dom.InfoSchema()) + require.NoError(t, err) + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats", `return(536870912)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume", `return(100)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/util/memory/ReadMemStats")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockAnalyzeMergeWorkerSlowConsume")) + }() + tk.MustQuery("select 1") + childTrackers = executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest() + require.Len(t, childTrackers, 0) + + h.HandleAutoAnalyze(dom.InfoSchema()) + rs := tk.MustQuery("select fail_reason from mysql.analyze_jobs where table_name=? and state=? limit 1", "t", "failed") + failReason := rs.Rows()[0][0].(string) + require.True(t, strings.Contains(failReason, "Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory. Please try narrowing your query scope or increase the tidb_server_memory_limit and try again.")) + + childTrackers = executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest() + require.Len(t, childTrackers, 0) +} + +func TestMemQuotaAnalyze(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table tbl_2 ( col_20 decimal default 84232 , col_21 tinyint not null , col_22 int default 80814394 , col_23 mediumint default -8036687 not null , col_24 smallint default 9185 not null , col_25 tinyint unsigned default 65 , col_26 char(115) default 'ZyfroRODMbNDRZnPNRW' not null , col_27 bigint not null , col_28 tinyint not null , col_29 char(130) default 'UMApsVgzHblmY' , primary key idx_14 ( col_28,col_22 ) , unique key idx_15 ( col_24,col_22 ) , key idx_16 ( col_21,col_20,col_24,col_25,col_27,col_28,col_26,col_29 ) , key idx_17 ( col_24,col_25 ) , unique key idx_18 ( col_25,col_23,col_29,col_27,col_26,col_22 ) , key idx_19 ( col_25,col_22,col_26,col_23 ) , unique key idx_20 ( col_22,col_24,col_28,col_29,col_26,col_20 ) , key idx_21 ( col_25,col_24,col_26,col_29,col_27,col_22,col_28 ) ) partition by range ( col_22 ) ( partition p0 values less than (-1938341588), partition p1 values less than (-1727506184), partition p2 values less than (-1700184882), partition p3 values less than (-1596142809), partition p4 values less than (445165686) );") + tk.MustExec("insert ignore into tbl_2 values ( 942,33,-1915007317,3408149,-3699,193,'Trywdis',1876334369465184864,115,null );") + tk.MustExec("insert ignore into tbl_2 values ( 7,-39,-1382727205,-2544981,-28075,88,'FDhOsTRKRLCwEk',-1239168882463214388,17,'WskQzCK' );") + tk.MustExec("insert ignore into tbl_2 values ( null,55,-388460319,-2292918,10130,162,'UqjDlYvdcNY',4872802276956896607,-51,'ORBQjnumcXP' );") + tk.MustExec("insert ignore into tbl_2 values ( 42,-19,-9677826,-1168338,16904,79,'TzOqH',8173610791128879419,65,'lNLcvOZDcRzWvDO' );") + tk.MustExec("insert ignore into tbl_2 values ( 2,26,369867543,-6773303,-24953,41,'BvbdrKTNtvBgsjjnxt',5996954963897924308,-95,'wRJYPBahkIGDfz' );") + tk.MustExec("insert ignore into tbl_2 values ( 6896,3,444460824,-2070971,-13095,167,'MvWNKbaOcnVuIrtbT',6968339995987739471,-5,'zWipNBxGeVmso' );") + tk.MustExec("insert ignore into tbl_2 values ( 58761,112,-1535034546,-5837390,-14204,157,'',-8319786912755096816,15,'WBjsozfBfrPPHmKv' );") + tk.MustExec("insert ignore into tbl_2 values ( 84923,113,-973946646,406140,25040,51,'THQdwkQvppWZnULm',5469507709881346105,94,'oGNmoxLLgHkdyDCT' );") + tk.MustExec("insert ignore into tbl_2 values ( 0,-104,-488745187,-1941015,-2646,39,'jyKxfs',-5307175470406648836,46,'KZpfjFounVgFeRPa' );") + tk.MustExec("insert ignore into tbl_2 values ( 4,97,2105289255,1034363,28385,192,'',4429378142102752351,8,'jOk' );") + tk.MustExec("set global tidb_mem_quota_analyze=128;") + tk.MustExecToErr("analyze table tbl_2;") +}