diff --git a/pkg/executor/analyze.go b/pkg/executor/analyze.go index bee9c08d2066d..eb889c546c0b9 100644 --- a/pkg/executor/analyze.go +++ b/pkg/executor/analyze.go @@ -16,6 +16,7 @@ package executor import ( "context" + stderrors "errors" "fmt" "math" "net" @@ -369,7 +370,17 @@ func (e *AnalyzeExec) handleResultsError( globalStatsMap globalStatsMap, resultsCh <-chan *statistics.AnalyzeResults, taskNum int, -) error { +) (err error) { + defer func() { + if r := recover(); r != nil { + logutil.BgLogger().Error("analyze save stats panic", zap.Any("recover", r), zap.Stack("stack")) + if err != nil { + err = stderrors.Join(err, getAnalyzePanicErr(r)) + } else { + err = getAnalyzePanicErr(r) + } + } + }() partitionStatsConcurrency := e.Ctx().GetSessionVars().AnalyzePartitionConcurrency // the concurrency of handleResultsError cannot be more than partitionStatsConcurrency partitionStatsConcurrency = min(taskNum, partitionStatsConcurrency) @@ -387,13 +398,12 @@ func (e *AnalyzeExec) handleResultsError( return err } } - + failpoint.Inject("handleResultsErrorSingleThreadPanic", nil) tableIDs := map[int64]struct{}{} // save analyze results in single-thread. statsHandle := domain.GetDomain(e.Ctx()).StatsHandle() panicCnt := 0 - var err error for panicCnt < concurrency { results, ok := <-resultsCh if !ok { diff --git a/pkg/executor/test/analyzetest/BUILD.bazel b/pkg/executor/test/analyzetest/BUILD.bazel index 5f37de422e445..2ca65d0d23eda 100644 --- a/pkg/executor/test/analyzetest/BUILD.bazel +++ b/pkg/executor/test/analyzetest/BUILD.bazel @@ -9,7 +9,7 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 48, + shard_count = 49, deps = [ "//pkg/config", "//pkg/domain", diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index 6fa02510e342b..62de329666a62 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -3118,3 +3118,18 @@ func TestAnalyzePartitionVerify(t *testing.T) { } } } + +func TestPanicInHandleResultErrorWithSingleGoroutine(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 ) );") + tk.MustExec("insert ignore into tbl_2 values ( 942,33,-1915007317,3408149,-3699,193,'Trywdis',1876334369465184864,115,null );") + fp := "github.com/pingcap/tidb/pkg/executor/handleResultsErrorSingleThreadPanic" + require.NoError(t, failpoint.Enable(fp, `panic("TestPanicInHandleResultErrorWithSingleGoroutine")`)) + defer func() { + require.NoError(t, failpoint.Disable(fp)) + }() + tk.MustExecToErr("analyze table tbl_2;") +}