diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 97531b2489140..c81efedb8958c 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -717,3 +717,22 @@ SELECT `).Check(testkit.Rows("t a b")) } } + +// https://github.com/pingcap/tidb/issues/36426. +func TestShowColumnsWithSubQueryView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("CREATE TABLE added (`id` int(11), `name` text, `some_date` timestamp);") + tk.MustExec("CREATE TABLE incremental (`id` int(11), `name`text, `some_date` timestamp);") + tk.MustExec("create view temp_view as (select * from `added` where id > (select max(id) from `incremental`));") + // Show columns should not send coprocessor request to the storage. + require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("timeout")`)) + tk.MustQuery("show columns from temp_view;").Check(testkit.Rows( + "id int(11) YES ", + "name text YES ", + "some_date timestamp YES ")) + require.NoError(t, failpoint.Disable("tikvclient/tikvStoreSendReqResult")) +} diff --git a/executor/show.go b/executor/show.go index 2f1d366dc7c1e..3439d83aca44f 100644 --- a/executor/show.go +++ b/executor/show.go @@ -2062,7 +2062,8 @@ func tryFillViewColumnType(ctx context.Context, sctx sessionctx.Context, is info // Take joining system table as an example, `fetchBuildSideRows` and `fetchProbeSideChunks` can be run concurrently. return runWithSystemSession(ctx, sctx, func(s sessionctx.Context) error { // Retrieve view columns info. - planBuilder, _ := plannercore.NewPlanBuilder().Init(s, is, &hint.BlockHintProcessor{}) + planBuilder, _ := plannercore.NewPlanBuilder( + plannercore.PlanBuilderOptNoExecution{}).Init(s, is, &hint.BlockHintProcessor{}) if viewLogicalPlan, err := planBuilder.BuildDataSourceFromView(ctx, dbName, tbl); err == nil { viewSchema := viewLogicalPlan.Schema() viewOutputNames := viewLogicalPlan.OutputNames() diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 9b27b112f0bda..381b4472d45a8 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -825,7 +825,8 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex return v, true } np = er.popExistsSubPlan(np) - if len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { + + if er.b.disableSubQueryPreprocessing || len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, v.Not, hasRewriteHint) if er.err != nil || !er.asScalar { return v, true @@ -1000,7 +1001,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S return v, true } np = er.b.buildMaxOneRow(np) - if len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { + if er.b.disableSubQueryPreprocessing || len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { er.p = er.b.buildApplyWithJoinType(er.p, np, LeftOuterJoin) if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b5c923ff6dd27..5c6aff3d2b53e 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -518,6 +518,8 @@ type PlanBuilder struct { // hasValidSemijoinHint would tell the outer APPLY/JOIN operator that there's valid hint to be checked later // if there's SEMI_JOIN_REWRITE hint and we find checkSemiJoinHint is true. hasValidSemiJoinHint bool + // disableSubQueryPreprocessing indicates whether to pre-process uncorrelated sub-queries in rewriting stage. + disableSubQueryPreprocessing bool } type handleColHelper struct { @@ -619,14 +621,31 @@ func (b *PlanBuilder) popSelectOffset() { b.selectOffset = b.selectOffset[:len(b.selectOffset)-1] } +// PlanBuilderOpt is used to adjust the plan builder. +type PlanBuilderOpt interface { + Apply(builder *PlanBuilder) +} + +// PlanBuilderOptNoExecution means the plan builder should not run any executor during Build(). +type PlanBuilderOptNoExecution struct{} + +// Apply implements the interface PlanBuilderOpt. +func (p PlanBuilderOptNoExecution) Apply(builder *PlanBuilder) { + builder.disableSubQueryPreprocessing = true +} + // NewPlanBuilder creates a new PlanBuilder. -func NewPlanBuilder() *PlanBuilder { - return &PlanBuilder{ +func NewPlanBuilder(opts ...PlanBuilderOpt) *PlanBuilder { + builder := &PlanBuilder{ outerCTEs: make([]*cteInfo, 0), colMapper: make(map[*ast.ColumnNameExpr]int), handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]HandleCols, 0)}, correlatedAggMapper: make(map[*ast.AggregateFuncExpr]*expression.CorrelatedColumn), } + for _, opt := range opts { + opt.Apply(builder) + } + return builder } // Init initialize a PlanBuilder.