Skip to content

Commit

Permalink
planner: fix the problem of not finding the referenced column in the …
Browse files Browse the repository at this point in the history
…virtual column (#57084) (#57293)

close #57079
  • Loading branch information
ti-chi-bot authored Nov 12, 2024
1 parent b36dc72 commit 0a3aa2a
Show file tree
Hide file tree
Showing 6 changed files with 101 additions and 8 deletions.
12 changes: 11 additions & 1 deletion pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3061,13 +3061,23 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) exec.Executor {
}
exprCtx := b.ctx.GetExprCtx()
for _, task := range v.ColTasks {
// ColumnInfos2ColumnsAndNames will use the `colInfos` to find the unique id for the column,
// so we need to make sure all the columns pass into it.
columns, _, err := expression.ColumnInfos2ColumnsAndNames(
exprCtx,
pmodel.NewCIStr(task.AnalyzeInfo.DBName),
task.TblInfo.Name,
task.ColsInfo,
append(task.ColsInfo, task.SkipColsInfo...),
task.TblInfo,
)
columns = slices.DeleteFunc(columns, func(expr *expression.Column) bool {
for _, col := range task.SkipColsInfo {
if col.ID == expr.ID {
return true
}
}
return false
})
if err != nil {
b.err = err
return nil
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -562,6 +562,7 @@ type AnalyzeColumnsTask struct {
HandleCols util.HandleCols
CommonHandleInfo *model.IndexInfo
ColsInfo []*model.ColumnInfo
SkipColsInfo []*model.ColumnInfo
TblInfo *model.TableInfo
Indexes []*model.IndexInfo
AnalyzeInfo
Expand Down
17 changes: 10 additions & 7 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2332,7 +2332,7 @@ func getModifiedIndexesInfoForAnalyze(
}

// filterSkipColumnTypes filters out columns whose types are in the skipTypes list.
func (b *PlanBuilder) filterSkipColumnTypes(origin []*model.ColumnInfo, tbl *resolve.TableNameW, mustAnalyzedCols *calcOnceMap) (result []*model.ColumnInfo) {
func (b *PlanBuilder) filterSkipColumnTypes(origin []*model.ColumnInfo, tbl *resolve.TableNameW, mustAnalyzedCols *calcOnceMap) (result []*model.ColumnInfo, skipCol []*model.ColumnInfo) {
// If the session is in restricted SQL mode, it uses @@global.tidb_analyze_skip_column_types to get the skipTypes list.
skipTypes := b.ctx.GetSessionVars().AnalyzeSkipColumnTypes
if b.ctx.GetSessionVars().InRestrictedSQL {
Expand Down Expand Up @@ -2362,6 +2362,7 @@ func (b *PlanBuilder) filterSkipColumnTypes(origin []*model.ColumnInfo, tbl *res
// into TiDB to build the index statistics.
_, keep := mustAnalyze[colInfo.ID]
if skip && !keep {
skipCol = append(skipCol, colInfo)
continue
}
result = append(result, colInfo)
Expand Down Expand Up @@ -2485,16 +2486,18 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask(
if colsInfo, ok := colsInfoMap[physicalID]; ok {
execColsInfo = colsInfo
}
execColsInfo = b.filterSkipColumnTypes(execColsInfo, tbl, &mustAnalyzedCols)
var skipColsInfo []*model.ColumnInfo
execColsInfo, skipColsInfo = b.filterSkipColumnTypes(execColsInfo, tbl, &mustAnalyzedCols)
allColumns := len(tbl.TableInfo.Columns) == len(execColsInfo)
indexes, independentIndexes, specialGlobalIndexes = getModifiedIndexesInfoForAnalyze(b.ctx.GetSessionVars().StmtCtx, tbl.TableInfo, allColumns, execColsInfo)
handleCols := BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo, allColumns, execColsInfo)
newTask := AnalyzeColumnsTask{
HandleCols: handleCols,
ColsInfo: execColsInfo,
AnalyzeInfo: info,
TblInfo: tbl.TableInfo,
Indexes: indexes,
HandleCols: handleCols,
ColsInfo: execColsInfo,
AnalyzeInfo: info,
TblInfo: tbl.TableInfo,
Indexes: indexes,
SkipColsInfo: skipColsInfo,
}
if newTask.HandleCols == nil {
extraCol := model.NewExtraHandleColInfo()
Expand Down
16 changes: 16 additions & 0 deletions pkg/planner/core/tests/analyze/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
load("@io_bazel_rules_go//go:def.bzl", "go_test")

go_test(
name = "analyze_test",
timeout = "short",
srcs = [
"analyze_test.go",
"main_test.go",
],
flaky = True,
deps = [
"//pkg/testkit",
"//pkg/testkit/testsetup",
"@org_uber_go_goleak//:goleak",
],
)
29 changes: 29 additions & 0 deletions pkg/planner/core/tests/analyze/analyze_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// Copyright 2024 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 analyze

import (
"testing"

"github.com/pingcap/tidb/pkg/testkit"
)

func TestAnalyzeVirtualColumns(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE t1 (id bigint NOT NULL,c1 varchar(50) NOT NULL ,c2 int DEFAULT NULL ,c3 json DEFAULT NULL ,c4 varchar(255) GENERATED ALWAYS AS (json_unquote(json_extract(c3, '$.oppositePlaceId'))) VIRTUAL ,PRIMARY KEY (id),UNIQUE KEY idx_unique (c1,c2)) ;`)
tk.MustExec("analyze table t1 all columns")
}
34 changes: 34 additions & 0 deletions pkg/planner/core/tests/analyze/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2024 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 analyze

import (
"flag"
"testing"

"github.com/pingcap/tidb/pkg/testkit/testsetup"
"go.uber.org/goleak"
)

func TestMain(m *testing.M) {
testsetup.SetupForCommonTest()
flag.Parse()
opts := []goleak.Option{
goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"),
goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"),
goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"),
}
goleak.VerifyTestMain(m, opts...)
}

0 comments on commit 0a3aa2a

Please sign in to comment.