diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index b1f9ba92be976..dcb09bf7de748 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -79,9 +79,9 @@ TopN_7 1.00 root test.t2.c2:asc, offset:0, count:1 └─TableScan_13 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t1 where c1 > 1 and c2 = 1 and c3 < 1; id count task operator info -IndexLookUp_11 1.11 root +IndexLookUp_11 11.08 root ├─IndexScan_8 33.33 cop[tikv] table:t1, index:c2, range:(1 1,1 +inf], keep order:false, stats:pseudo -└─Selection_10 1.11 cop[tikv] lt(test.t1.c3, 1) +└─Selection_10 11.08 cop[tikv] lt(test.t1.c3, 1) └─TableScan_9 33.33 cop[tikv] table:t1, keep order:false, stats:pseudo explain select * from t1 where c1 = 1 and c2 > 1; id count task operator info diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index c327f9bcd6747..4fabc86e74c40 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1043,6 +1043,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) ticker := time.NewTicker(s.lease) + defer ticker.Stop() for range ticker.C { historyJob, err := s.getHistoryDDLJob(job.ID) c.Assert(err, IsNil) diff --git a/domain/schema_validator_test.go b/domain/schema_validator_test.go index 08b6aae000e20..7ddc97654aef2 100644 --- a/domain/schema_validator_test.go +++ b/domain/schema_validator_test.go @@ -128,6 +128,7 @@ func serverFunc(lease time.Duration, requireLease chan leaseGrantItem, oracleCh var version int64 leaseTS := uint64(time.Now().UnixNano()) ticker := time.NewTicker(lease) + defer ticker.Stop() for { select { case now := <-ticker.C: diff --git a/executor/adapter.go b/executor/adapter.go index 9e2cb5f3134c5..4e6b47d3d84d6 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -254,7 +254,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { - is := GetInfoSchema(a.Ctx) + is := infoschema.GetInfoSchema(a.Ctx) a.InfoSchema = is if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { return 0, err diff --git a/executor/analyze.go b/executor/analyze.go index 7d91bdaff42d9..f6145495578f3 100755 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" @@ -125,7 +126,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - return statsHandle.Update(GetInfoSchema(e.ctx)) + return statsHandle.Update(infoschema.GetInfoSchema(e.ctx)) } func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { diff --git a/executor/analyze_test.go b/executor/analyze_test.go index e0e1dbc817424..c1a6a6965e75e 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -62,7 +63,7 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("analyze table t") - is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() @@ -89,7 +90,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") - is = executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + is = infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi = table.Meta().GetPartitionInfo() @@ -139,7 +140,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t with 30 samples") - is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -170,7 +171,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") - is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -311,7 +312,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } tk.MustExec("analyze table t with 5 buckets, 6 samples") - is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() diff --git a/executor/bind.go b/executor/bind.go index ef160bd868f70..b2ef276342827 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/infoschema" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/util/chunk" ) @@ -72,9 +73,9 @@ func (e *SQLBindExec) dropSQLBind() error { } if !e.isGlobal { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - return handle.DropBindRecord(e.ctx, GetInfoSchema(e.ctx), record) + return handle.DropBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record) } - return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.ctx, GetInfoSchema(e.ctx), record) + return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record) } func (e *SQLBindExec) createSQLBind() error { @@ -91,9 +92,9 @@ func (e *SQLBindExec) createSQLBind() error { } if !e.isGlobal { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) - return handle.AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record) + return handle.AddBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record) } - return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record) + return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record) } func (e *SQLBindExec) flushBindings() error { diff --git a/executor/compiler.go b/executor/compiler.go index e55d7d04b5945..b7077e066d8ec 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -25,8 +25,6 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" ) var ( @@ -55,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } - infoSchema := GetInfoSchema(c.Ctx) + infoSchema := infoschema.GetInfoSchema(c.Ctx) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err } @@ -347,17 +345,3 @@ func GetStmtLabel(stmtNode ast.StmtNode) string { } return "other" } - -// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set, -// Otherwise, snapshot schema is returned. -func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema { - sessVar := ctx.GetSessionVars() - var is infoschema.InfoSchema - if snap := sessVar.SnapshotInfoschema; snap != nil { - is = snap.(infoschema.InfoSchema) - logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion())) - } else { - is = sessVar.TxnCtx.InfoSchema.(infoschema.InfoSchema) - } - return is -} diff --git a/executor/executor.go b/executor/executor.go index 6a10c3620642d..d11969fa58e61 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1618,14 +1618,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.PrevAffectedRows = -1 } errCount, warnCount := vars.StmtCtx.NumErrorWarnings() - err = vars.SetSystemVar("warning_count", warnCount) - if err != nil { - return err - } - err = vars.SetSystemVar("error_count", errCount) - if err != nil { - return err - } + vars.SysErrorCount = errCount + vars.SysWarningCount = warnCount vars.StmtCtx = sc for _, warn := range hintWarns { vars.StmtCtx.AppendWarning(warn) diff --git a/executor/executor_test.go b/executor/executor_test.go index 6294973172c02..8fb7d74f28a1d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2089,7 +2089,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } - infoSchema := executor.GetInfoSchema(ctx) + infoSchema := infoschema.GetInfoSchema(ctx) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") diff --git a/executor/load_stats.go b/executor/load_stats.go index 58a764748341e..83fbb3ad188f7 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" @@ -85,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error { if h == nil { return errors.New("Load Stats: handle is nil") } - return h.LoadStatsFromJSON(GetInfoSchema(e.Ctx), jsonTbl) + return h.LoadStatsFromJSON(infoschema.GetInfoSchema(e.Ctx), jsonTbl) } diff --git a/executor/metrics_test.go b/executor/metrics_test.go index ba6f3cc866738..19000b4faee5d 100644 --- a/executor/metrics_test.go +++ b/executor/metrics_test.go @@ -20,6 +20,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" @@ -61,7 +62,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) - is := executor.GetInfoSchema(tk.Se) + is := infoschema.GetInfoSchema(tk.Se) err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) diff --git a/executor/prepared.go b/executor/prepared.go index dd8152341929f..8c84b82e88333 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -297,7 +297,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, return nil, err } execStmt.BinaryArgs = args - is := GetInfoSchema(sctx) + is := infoschema.GetInfoSchema(sctx) execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is) if err != nil { return nil, err diff --git a/executor/show.go b/executor/show.go index 04dfec1326e7d..cd19643cf0b0e 100644 --- a/executor/show.go +++ b/executor/show.go @@ -39,9 +39,11 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" @@ -345,7 +347,7 @@ func (e *ShowExec) fetchShowTableStatus() error { FROM information_schema.tables WHERE table_schema='%s' ORDER BY table_name`, e.DBName) - rows, _, err := e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) + rows, _, err := e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQLWithSnapshot(sql) if err != nil { return errors.Trace(err) @@ -632,53 +634,45 @@ func escape(cis model.CIStr, sqlMode mysql.SQLMode) string { return quote + strings.Replace(cis.O, quote, quote+quote, -1) + quote } -func (e *ShowExec) fetchShowCreateTable() error { - tb, err := e.getTable() - if err != nil { - return errors.Trace(err) - } - - sqlMode := e.ctx.GetSessionVars().SQLMode - - // TODO: let the result more like MySQL. - var buf bytes.Buffer - if tb.Meta().IsView() { - e.fetchShowCreateTable4View(tb.Meta(), &buf) - e.appendRow([]interface{}{tb.Meta().Name.O, buf.String(), tb.Meta().Charset, tb.Meta().Collate}) +// ConstructResultOfShowCreateTable constructs the result for show create table. +func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.TableInfo, allocator autoid.Allocator, buf *bytes.Buffer) (err error) { + if tableInfo.IsView() { + fetchShowCreateTable4View(ctx, tableInfo, buf) return nil } - tblCharset := tb.Meta().Charset + tblCharset := tableInfo.Charset if len(tblCharset) == 0 { tblCharset = mysql.DefaultCharset } - tblCollate := tb.Meta().Collate + tblCollate := tableInfo.Collate // Set default collate if collate is not specified. if len(tblCollate) == 0 { tblCollate = getDefaultCollate(tblCharset) } - fmt.Fprintf(&buf, "CREATE TABLE %s (\n", escape(tb.Meta().Name, sqlMode)) - var pkCol *table.Column + sqlMode := ctx.GetSessionVars().SQLMode + fmt.Fprintf(buf, "CREATE TABLE %s (\n", escape(tableInfo.Name, sqlMode)) + var pkCol *model.ColumnInfo var hasAutoIncID bool - for i, col := range tb.Cols() { - fmt.Fprintf(&buf, " %s %s", escape(col.Name, sqlMode), col.GetTypeDesc()) + for i, col := range tableInfo.Cols() { + fmt.Fprintf(buf, " %s %s", escape(col.Name, sqlMode), col.GetTypeDesc()) if col.Charset != "binary" { if col.Charset != tblCharset { - fmt.Fprintf(&buf, " CHARACTER SET %s", col.Charset) + fmt.Fprintf(buf, " CHARACTER SET %s", col.Charset) } if col.Collate != tblCollate { - fmt.Fprintf(&buf, " COLLATE %s", col.Collate) + fmt.Fprintf(buf, " COLLATE %s", col.Collate) } else { defcol, err := charset.GetDefaultCollation(col.Charset) if err == nil && defcol != col.Collate { - fmt.Fprintf(&buf, " COLLATE %s", col.Collate) + fmt.Fprintf(buf, " COLLATE %s", col.Collate) } } } if col.IsGenerated() { // It's a generated column. - fmt.Fprintf(&buf, " GENERATED ALWAYS AS (%s)", col.GeneratedExprString) + fmt.Fprintf(buf, " GENERATED ALWAYS AS (%s)", col.GeneratedExprString) if col.GeneratedStored { buf.WriteString(" STORED") } else { @@ -712,7 +706,7 @@ func (e *ShowExec) fetchShowCreateTable() error { defaultValStr := fmt.Sprintf("%v", defaultValue) // If column is timestamp, and default value is not current_timestamp, should convert the default value to the current session time zone. if col.Tp == mysql.TypeTimestamp && defaultValStr != types.ZeroDatetimeStr { - timeValue, err := table.GetColDefaultValue(e.ctx, col.ToInfo()) + timeValue, err := table.GetColDefaultValue(ctx, col) if err != nil { return errors.Trace(err) } @@ -721,9 +715,9 @@ func (e *ShowExec) fetchShowCreateTable() error { if col.Tp == mysql.TypeBit { defaultValBinaryLiteral := types.BinaryLiteral(defaultValStr) - fmt.Fprintf(&buf, " DEFAULT %s", defaultValBinaryLiteral.ToBitLiteralString(true)) + fmt.Fprintf(buf, " DEFAULT %s", defaultValBinaryLiteral.ToBitLiteralString(true)) } else { - fmt.Fprintf(&buf, " DEFAULT '%s'", format.OutputFormat(defaultValStr)) + fmt.Fprintf(buf, " DEFAULT '%s'", format.OutputFormat(defaultValStr)) } } } @@ -733,12 +727,12 @@ func (e *ShowExec) fetchShowCreateTable() error { } } if len(col.Comment) > 0 { - fmt.Fprintf(&buf, " COMMENT '%s'", format.OutputFormat(col.Comment)) + fmt.Fprintf(buf, " COMMENT '%s'", format.OutputFormat(col.Comment)) } - if i != len(tb.Cols())-1 { + if i != len(tableInfo.Cols())-1 { buf.WriteString(",\n") } - if tb.Meta().PKIsHandle && mysql.HasPriKeyFlag(col.Flag) { + if tableInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag) { pkCol = col } } @@ -746,12 +740,12 @@ func (e *ShowExec) fetchShowCreateTable() error { if pkCol != nil { // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") - fmt.Fprintf(&buf, " PRIMARY KEY (%s)", escape(pkCol.Name, sqlMode)) + fmt.Fprintf(buf, " PRIMARY KEY (%s)", escape(pkCol.Name, sqlMode)) } - publicIndices := make([]table.Index, 0, len(tb.Indices())) - for _, idx := range tb.Indices() { - if idx.Meta().State == model.StatePublic { + publicIndices := make([]*model.IndexInfo, 0, len(tableInfo.Indices)) + for _, idx := range tableInfo.Indices { + if idx.State == model.StatePublic { publicIndices = append(publicIndices, idx) } } @@ -759,14 +753,13 @@ func (e *ShowExec) fetchShowCreateTable() error { buf.WriteString(",\n") } - for i, idx := range publicIndices { - idxInfo := idx.Meta() + for i, idxInfo := range publicIndices { if idxInfo.Primary { buf.WriteString(" PRIMARY KEY ") } else if idxInfo.Unique { - fmt.Fprintf(&buf, " UNIQUE KEY %s ", escape(idxInfo.Name, sqlMode)) + fmt.Fprintf(buf, " UNIQUE KEY %s ", escape(idxInfo.Name, sqlMode)) } else { - fmt.Fprintf(&buf, " KEY %s ", escape(idxInfo.Name, sqlMode)) + fmt.Fprintf(buf, " KEY %s ", escape(idxInfo.Name, sqlMode)) } cols := make([]string, 0, len(idxInfo.Columns)) @@ -777,7 +770,7 @@ func (e *ShowExec) fetchShowCreateTable() error { } cols = append(cols, colInfo) } - fmt.Fprintf(&buf, "(%s)", strings.Join(cols, ",")) + fmt.Fprintf(buf, "(%s)", strings.Join(cols, ",")) if i != len(publicIndices)-1 { buf.WriteString(",\n") } @@ -791,40 +784,60 @@ func (e *ShowExec) fetchShowCreateTable() error { if len(tblCollate) == 0 { // If we can not find default collate for the given charset, // do not show the collate part. - fmt.Fprintf(&buf, " DEFAULT CHARSET=%s", tblCharset) + fmt.Fprintf(buf, " DEFAULT CHARSET=%s", tblCharset) } else { - fmt.Fprintf(&buf, " DEFAULT CHARSET=%s COLLATE=%s", tblCharset, tblCollate) + fmt.Fprintf(buf, " DEFAULT CHARSET=%s COLLATE=%s", tblCharset, tblCollate) } // Displayed if the compression typed is set. - if len(tb.Meta().Compression) != 0 { - fmt.Fprintf(&buf, " COMPRESSION='%s'", tb.Meta().Compression) + if len(tableInfo.Compression) != 0 { + fmt.Fprintf(buf, " COMPRESSION='%s'", tableInfo.Compression) } if hasAutoIncID { - autoIncID, err := tb.Allocator(e.ctx).NextGlobalAutoID(tb.Meta().ID) + autoIncID, err := allocator.NextGlobalAutoID(tableInfo.ID) if err != nil { return errors.Trace(err) } - // It's campatible with MySQL. + // It's compatible with MySQL. if autoIncID > 1 { - fmt.Fprintf(&buf, " AUTO_INCREMENT=%d", autoIncID) + fmt.Fprintf(buf, " AUTO_INCREMENT=%d", autoIncID) } } - if tb.Meta().ShardRowIDBits > 0 { - fmt.Fprintf(&buf, "/*!90000 SHARD_ROW_ID_BITS=%d ", tb.Meta().ShardRowIDBits) - if tb.Meta().PreSplitRegions > 0 { - fmt.Fprintf(&buf, "PRE_SPLIT_REGIONS=%d ", tb.Meta().PreSplitRegions) + if tableInfo.ShardRowIDBits > 0 { + fmt.Fprintf(buf, "/*!90000 SHARD_ROW_ID_BITS=%d ", tableInfo.ShardRowIDBits) + if tableInfo.PreSplitRegions > 0 { + fmt.Fprintf(buf, "PRE_SPLIT_REGIONS=%d ", tableInfo.PreSplitRegions) } buf.WriteString("*/") } - if len(tb.Meta().Comment) > 0 { - fmt.Fprintf(&buf, " COMMENT='%s'", format.OutputFormat(tb.Meta().Comment)) + if len(tableInfo.Comment) > 0 { + fmt.Fprintf(buf, " COMMENT='%s'", format.OutputFormat(tableInfo.Comment)) } // add partition info here. - appendPartitionInfo(tb.Meta().Partition, &buf) + appendPartitionInfo(tableInfo.Partition, buf) + return nil +} + +func (e *ShowExec) fetchShowCreateTable() error { + tb, err := e.getTable() + if err != nil { + return errors.Trace(err) + } + + tableInfo := tb.Meta() + allocator := tb.Allocator(e.ctx) + var buf bytes.Buffer + // TODO: let the result more like MySQL. + if err = ConstructResultOfShowCreateTable(e.ctx, tb.Meta(), allocator, &buf); err != nil { + return err + } + if tableInfo.IsView() { + e.appendRow([]interface{}{tableInfo.Name.O, buf.String(), tableInfo.Charset, tableInfo.Collate}) + return nil + } e.appendRow([]interface{}{tb.Meta().Name.O, buf.String()}) return nil @@ -846,13 +859,13 @@ func (e *ShowExec) fetchShowCreateView() error { } var buf bytes.Buffer - e.fetchShowCreateTable4View(tb.Meta(), &buf) + fetchShowCreateTable4View(e.ctx, tb.Meta(), &buf) e.appendRow([]interface{}{tb.Meta().Name.O, buf.String(), tb.Meta().Charset, tb.Meta().Collate}) return nil } -func (e *ShowExec) fetchShowCreateTable4View(tb *model.TableInfo, buf *bytes.Buffer) { - sqlMode := e.ctx.GetSessionVars().SQLMode +func fetchShowCreateTable4View(ctx sessionctx.Context, tb *model.TableInfo, buf *bytes.Buffer) { + sqlMode := ctx.GetSessionVars().SQLMode fmt.Fprintf(buf, "CREATE ALGORITHM=%s ", tb.View.Algorithm.String()) fmt.Fprintf(buf, "DEFINER=%s@%s ", escape(model.NewCIStr(tb.View.Definer.Username), sqlMode), escape(model.NewCIStr(tb.View.Definer.Hostname), sqlMode)) @@ -901,6 +914,20 @@ func appendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer) buf.WriteString(")") } +// ConstructResultOfShowCreateDatabase constructs the result for show create database. +func ConstructResultOfShowCreateDatabase(ctx sessionctx.Context, dbInfo *model.DBInfo, ifNotExists bool, buf *bytes.Buffer) (err error) { + sqlMode := ctx.GetSessionVars().SQLMode + var ifNotExistsStr string + if ifNotExists { + ifNotExistsStr = "/*!32312 IF NOT EXISTS*/ " + } + fmt.Fprintf(buf, "CREATE DATABASE %s%s", ifNotExistsStr, escape(dbInfo.Name, sqlMode)) + if s := dbInfo.Charset; len(s) > 0 { + fmt.Fprintf(buf, " /*!40100 DEFAULT CHARACTER SET %s */", s) + } + return nil +} + // fetchShowCreateDatabase composes show create database result. func (e *ShowExec) fetchShowCreateDatabase() error { checker := privilege.GetPrivilegeManager(e.ctx) @@ -909,24 +936,17 @@ func (e *ShowExec) fetchShowCreateDatabase() error { return e.dbAccessDenied() } } - db, ok := e.is.SchemaByName(e.DBName) + dbInfo, ok := e.is.SchemaByName(e.DBName) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(e.DBName.O) } - sqlMode := e.ctx.GetSessionVars().SQLMode - var buf bytes.Buffer - var ifNotExists string - if e.IfNotExists { - ifNotExists = "/*!32312 IF NOT EXISTS*/ " - } - fmt.Fprintf(&buf, "CREATE DATABASE %s%s", ifNotExists, escape(db.Name, sqlMode)) - if s := db.Charset; len(s) > 0 { - fmt.Fprintf(&buf, " /*!40100 DEFAULT CHARACTER SET %s */", s) + err := ConstructResultOfShowCreateDatabase(e.ctx, dbInfo, e.IfNotExists, &buf) + if err != nil { + return err } - - e.appendRow([]interface{}{db.Name.O, buf.String()}) + e.appendRow([]interface{}{dbInfo.Name.O, buf.String()}) return nil } diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index 152898eb2170d..1aae306c9ef7a 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -14,6 +14,9 @@ package executor_test import ( + "fmt" + "time" + . "github.com/pingcap/check" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/testkit" @@ -203,3 +206,28 @@ func (s *testShowStatsSuite) TestShowAnalyzeStatus(c *C) { c.Assert(result.Rows()[1][5], NotNil) c.Assert(result.Rows()[1][6], Equals, "finished") } + +func (s *testShowStatsSuite) TestShowStatusSnapshot(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("drop database if exists test;") + tk.MustExec("create database test;") + tk.MustExec("use test;") + tk.MustExec("create table t (a int);") + + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20060102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + snapshotTime := time.Now() + + tk.MustExec("drop table t;") + tk.MustQuery("show table status;").Check(testkit.Rows()) + tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'") + result := tk.MustQuery("show table status;") + c.Check(result.Rows()[0][0], Matches, "t") +} diff --git a/executor/simple.go b/executor/simple.go index 5952bf2491524..aea375ab54e85 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1048,7 +1048,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) error { if err != nil { return err } - return h.Update(GetInfoSchema(e.ctx)) + return h.Update(infoschema.GetInfoSchema(e.ctx)) } func (e *SimpleExec) autoNewTxn() bool { diff --git a/expression/builtin.go b/expression/builtin.go index 001f91e471597..9b192d2d2bb29 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" @@ -48,6 +49,9 @@ type baseBuiltinFunc struct { childrenVectorizedOnce *sync.Once childrenVectorized bool + + childrenReversedOnce *sync.Once + childrenReversed bool } func (b *baseBuiltinFunc) PbCode() tipb.ScalarFuncSig { @@ -74,6 +78,7 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, args []Expression) baseBuiltinFu return baseBuiltinFunc{ bufAllocator: newLocalSliceBuffer(len(args)), childrenVectorizedOnce: new(sync.Once), + childrenReversedOnce: new(sync.Once), args: args, ctx: ctx, @@ -179,6 +184,7 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, args []Expression, retType return baseBuiltinFunc{ bufAllocator: newLocalSliceBuffer(len(args)), childrenVectorizedOnce: new(sync.Once), + childrenReversedOnce: new(sync.Once), args: args, ctx: ctx, @@ -250,6 +256,27 @@ func (b *baseBuiltinFunc) vectorized() bool { return false } +func (b *baseBuiltinFunc) supportReverseEval() bool { + return false +} + +func (b *baseBuiltinFunc) isChildrenReversed() bool { + b.childrenReversedOnce.Do(func() { + b.childrenReversed = true + for _, arg := range b.args { + if !arg.SupportReverseEval() { + b.childrenReversed = false + break + } + } + }) + return b.childrenReversed +} + +func (b *baseBuiltinFunc) reverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (types.Datum, error) { + return types.Datum{}, errors.Errorf("baseBuiltinFunc.reverseEvalInt() should never be called, please contact the TiDB team for help") +} + func (b *baseBuiltinFunc) isChildrenVectorized() bool { b.childrenVectorizedOnce.Do(func() { b.childrenVectorized = true @@ -305,6 +332,7 @@ func (b *baseBuiltinFunc) cloneFrom(from *baseBuiltinFunc) { b.pbCode = from.pbCode b.bufAllocator = newLocalSliceBuffer(len(b.args)) b.childrenVectorizedOnce = new(sync.Once) + b.childrenReversedOnce = new(sync.Once) } func (b *baseBuiltinFunc) Clone() builtinFunc { @@ -372,9 +400,22 @@ type vecBuiltinFunc interface { vecEvalJSON(input *chunk.Chunk, result *chunk.Column) error } +// reverseBuiltinFunc evaluates the exactly one column value in the function when given a result for expression. +// For example, the buitinFunc is builtinArithmeticPlusRealSig(2.3, builtinArithmeticMinusRealSig(Column, 3.4)) +// when given the result like 1.0, then the ReverseEval should evaluate the column value 1.0 - 2.3 + 3.4 = 2.1 +type reverseBuiltinFunc interface { + // supportReverseEval checks whether the builtinFunc support reverse evaluation. + supportReverseEval() bool + // isChildrenReversed checks whether the builtinFunc's children support reverse evaluation. + isChildrenReversed() bool + // reverseEval evaluates the only one column value with given function result. + reverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) +} + // builtinFunc stands for a particular function signature. type builtinFunc interface { vecBuiltinFunc + reverseBuiltinFunc // evalInt evaluates int result of builtinFunc by given row. evalInt(row chunk.Row) (val int64, isNull bool, err error) diff --git a/expression/builtin_info_vec.go b/expression/builtin_info_vec.go index 42638fcf76c9f..7b26f448ba023 100644 --- a/expression/builtin_info_vec.go +++ b/expression/builtin_info_vec.go @@ -191,11 +191,22 @@ func (b *builtinTiDBIsDDLOwnerSig) vecEvalInt(input *chunk.Chunk, result *chunk. } func (b *builtinFoundRowsSig) vectorized() bool { - return false + return true } func (b *builtinFoundRowsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + data := b.ctx.GetSessionVars() + if data == nil { + return errors.Errorf("Missing session variable when eval builtin") + } + lastFoundRows := int64(data.LastFoundRows) + n := input.NumRows() + result.ResizeInt64(n, false) + i64s := result.Int64s() + for i := range i64s { + i64s[i] = lastFoundRows + } + return nil } func (b *builtinBenchmarkSig) vectorized() bool { diff --git a/expression/builtin_info_vec_test.go b/expression/builtin_info_vec_test.go index 017dc75fac36b..bd3d43a8d365c 100644 --- a/expression/builtin_info_vec_test.go +++ b/expression/builtin_info_vec_test.go @@ -53,7 +53,9 @@ var vecBuiltinInfoCases = map[string][]vecExprBenchCase{ ast.CurrentUser: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{}}, }, - ast.FoundRows: {}, + ast.FoundRows: { + {retEvalType: types.ETInt}, + }, ast.Database: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{}}, }, diff --git a/expression/builtin_json.go b/expression/builtin_json.go index aefbe93d7421e..0d7844ec0264f 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -15,6 +15,7 @@ package expression import ( json2 "encoding/json" + "strconv" "strings" "github.com/pingcap/errors" @@ -182,11 +183,21 @@ func (b *builtinJSONUnquoteSig) Clone() builtinFunc { return newSig } +func (c *jsonUnquoteFunctionClass) verifyArgs(args []Expression) error { + if err := c.baseFunctionClass.verifyArgs(args); err != nil { + return err + } + if evalType := args[0].GetType().EvalType(); evalType != types.ETString && evalType != types.ETJson { + return ErrIncorrectType.GenWithStackByArgs("1", "json_unquote") + } + return nil +} + func (c *jsonUnquoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETJson) + bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) bf.tp.Flen = mysql.MaxFieldVarCharLength DisableParseJSONFlag4Expr(args[0]) sig := &builtinJSONUnquoteSig{bf} @@ -194,14 +205,16 @@ func (c *jsonUnquoteFunctionClass) getFunction(ctx sessionctx.Context, args []Ex return sig, nil } -func (b *builtinJSONUnquoteSig) evalString(row chunk.Row) (res string, isNull bool, err error) { - var j json.BinaryJSON - j, isNull, err = b.args[0].EvalJSON(b.ctx, row) +func (b *builtinJSONUnquoteSig) evalString(row chunk.Row) (string, bool, error) { + str, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { return "", isNull, err } - res, err = j.Unquote() - return res, err != nil, err + str, err = json.UnquoteString(str) + if err != nil { + return "", false, err + } + return str, false, nil } type jsonSetFunctionClass struct { @@ -1022,24 +1035,33 @@ func (b *builtinJSONQuoteSig) Clone() builtinFunc { return newSig } +func (c *jsonQuoteFunctionClass) verifyArgs(args []Expression) error { + if err := c.baseFunctionClass.verifyArgs(args); err != nil { + return err + } + if evalType := args[0].GetType().EvalType(); evalType != types.ETString { + return ErrIncorrectType.GenWithStackByArgs("1", "json_quote") + } + return nil +} + func (c *jsonQuoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETJson) + bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) DisableParseJSONFlag4Expr(args[0]) sig := &builtinJSONQuoteSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonQuoteSig) return sig, nil } -func (b *builtinJSONQuoteSig) evalString(row chunk.Row) (res string, isNull bool, err error) { - var j json.BinaryJSON - j, isNull, err = b.args[0].EvalJSON(b.ctx, row) +func (b *builtinJSONQuoteSig) evalString(row chunk.Row) (string, bool, error) { + str, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { return "", isNull, err } - return j.Quote(), false, nil + return strconv.Quote(str), false, nil } type jsonSearchFunctionClass struct { diff --git a/expression/builtin_json_vec.go b/expression/builtin_json_vec.go index 903a15e82cf6e..53db3b56ab779 100644 --- a/expression/builtin_json_vec.go +++ b/expression/builtin_json_vec.go @@ -14,6 +14,8 @@ package expression import ( + "strconv" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/sessionctx" @@ -299,12 +301,12 @@ func (b *builtinJSONQuoteSig) vectorized() bool { func (b *builtinJSONQuoteSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { n := input.NumRows() - buf, err := b.bufAllocator.get(types.ETJson, n) + buf, err := b.bufAllocator.get(types.ETString, n) if err != nil { return err } defer b.bufAllocator.put(buf) - if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil { + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } @@ -314,7 +316,7 @@ func (b *builtinJSONQuoteSig) vecEvalString(input *chunk.Chunk, result *chunk.Co result.AppendNull() continue } - result.AppendString(buf.GetJSON(i).Quote()) + result.AppendString(strconv.Quote(buf.GetString(i))) } return nil } @@ -811,12 +813,12 @@ func (b *builtinJSONUnquoteSig) vectorized() bool { func (b *builtinJSONUnquoteSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { n := input.NumRows() - buf, err := b.bufAllocator.get(types.ETJson, n) + buf, err := b.bufAllocator.get(types.ETString, n) if err != nil { return err } defer b.bufAllocator.put(buf) - if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil { + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } @@ -826,11 +828,11 @@ func (b *builtinJSONUnquoteSig) vecEvalString(input *chunk.Chunk, result *chunk. result.AppendNull() continue } - res, err := buf.GetJSON(i).Unquote() + str, err := json.UnquoteString(buf.GetString(i)) if err != nil { return err } - result.AppendString(res) + result.AppendString(str) } return nil } diff --git a/expression/builtin_json_vec_test.go b/expression/builtin_json_vec_test.go index 9640330abaa2f..5b0ffb4fb5fc1 100644 --- a/expression/builtin_json_vec_test.go +++ b/expression/builtin_json_vec_test.go @@ -102,7 +102,7 @@ var vecBuiltinJSONCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson, types.ETString, types.ETJson}, geners: []dataGenerator{nil, &constStrGener{"$.aaa"}, nil, &constStrGener{"$.bbb"}, nil}}, }, ast.JSONQuote: { - {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETJson}}, + {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}}, }, } diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 66ece06697950..7f60dfe591153 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -1770,7 +1770,7 @@ func (c *truncateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre } argTp := args[0].GetType().EvalType() - if argTp == types.ETTimestamp || argTp == types.ETDatetime || argTp == types.ETDuration || argTp == types.ETString { + if argTp.IsStringKind() { argTp = types.ETReal } @@ -1796,6 +1796,8 @@ func (c *truncateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre case types.ETDecimal: sig = &builtinTruncateDecimalSig{bf} sig.setPbCode(tipb.ScalarFuncSig_TruncateDecimal) + default: + return nil, errIncorrectArgs.GenWithStackByArgs("truncate") } return sig, nil diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index 3f48e01fee73f..2263b8495054a 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -1175,11 +1175,62 @@ func (b *builtinLocateBinary2ArgsSig) vecEvalInt(input *chunk.Chunk, result *chu } func (b *builtinLocateBinary3ArgsSig) vectorized() bool { - return false + return true } +// vecEvalInt evals LOCATE(substr,str,pos), case-sensitive. +// See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_locate func (b *builtinLocateBinary3ArgsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + buf0, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEvalString(b.ctx, input, buf0); err != nil { + return err + } + buf1, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { + return err + } + // store positions in result + if err := b.args[2].VecEvalInt(b.ctx, input, result); err != nil { + return err + } + + result.MergeNulls(buf0, buf1) + i64s := result.Int64s() + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + pos := i64s[i] + // Transfer the argument which starts from 1 to real index which starts from 0. + pos-- + subStr := buf0.GetString(i) + str := buf1.GetString(i) + subStrLen := len(subStr) + if pos < 0 || pos > int64(len(str)-subStrLen) { + i64s[i] = 0 + continue + } else if subStrLen == 0 { + i64s[i] = pos + 1 + continue + } + slice := str[pos:] + idx := strings.Index(slice, subStr) + if idx != -1 { + i64s[i] = pos + int64(idx) + 1 + continue + } + i64s[i] = 0 + } + return nil } func (b *builtinExportSet4ArgSig) vectorized() bool { diff --git a/expression/builtin_string_vec_test.go b/expression/builtin_string_vec_test.go index 75a1faaece68e..192328e8aa7f4 100644 --- a/expression/builtin_string_vec_test.go +++ b/expression/builtin_string_vec_test.go @@ -191,6 +191,18 @@ var vecBuiltinStringCases = map[string][]vecExprBenchCase{ childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}, {Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}}, geners: []dataGenerator{&selectStringGener{[]string{"01", "10", "001", "110", "0001", "1110"}}, &selectStringGener{[]string{"010010001000010", "101101110111101"}}}, }, + { + retEvalType: types.ETInt, + childrenTypes: []types.EvalType{types.ETString, types.ETString, types.ETInt}, + childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}, {Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}, {Tp: mysql.TypeInt24}}, + geners: []dataGenerator{&randLenStrGener{0, 10}, &randLenStrGener{0, 20}, &rangeInt64Gener{-10, 20}}, + }, + { + retEvalType: types.ETInt, + childrenTypes: []types.EvalType{types.ETString, types.ETString, types.ETInt}, + childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}, {Tp: mysql.TypeString, Flag: mysql.BinaryFlag, Collate: charset.CollationBin}, {Tp: mysql.TypeInt24}}, + geners: []dataGenerator{&selectStringGener{[]string{"01", "10", "001", "110", "0001", "1110"}}, &selectStringGener{[]string{"010010001000010", "101101110111101"}}, &rangeInt64Gener{-10, 20}}, + }, }, ast.Hex: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&randHexStrGener{10, 100}}}, diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index af0fa8ce25ae9..dd0d991a1b715 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -147,13 +147,15 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ ast.SecToTime: { {retEvalType: types.ETDuration, childrenTypes: []types.EvalType{types.ETReal}}, }, - ast.TimestampAdd: { - { - retEvalType: types.ETString, - childrenTypes: []types.EvalType{types.ETString, types.ETInt, types.ETDatetime}, - geners: []dataGenerator{&unitStrGener{}, nil, nil}, - }, - }, + // This test case may fail due to the issue: https://github.com/pingcap/tidb/issues/13638. + // We remove this case to stabilize CI, and will reopen this when we fix the issue above. + //ast.TimestampAdd: { + // { + // retEvalType: types.ETString, + // childrenTypes: []types.EvalType{types.ETString, types.ETInt, types.ETDatetime}, + // geners: []dataGenerator{&unitStrGener{}, nil, nil}, + // }, + //}, ast.TimestampDiff: { { retEvalType: types.ETInt, diff --git a/expression/column.go b/expression/column.go index 8b0fa3f96d3b8..98601053c328d 100644 --- a/expression/column.go +++ b/expression/column.go @@ -581,3 +581,18 @@ idLoop: func (col *Column) EvalVirtualColumn(row chunk.Row) (types.Datum, error) { return col.VirtualExpr.Eval(row) } + +// SupportReverseEval checks whether the builtinFunc support reverse evaluation. +func (col *Column) SupportReverseEval() bool { + switch col.RetType.Tp { + case mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, + mysql.TypeFloat, mysql.TypeDouble, mysql.TypeNewDecimal: + return true + } + return false +} + +// ReverseEval evaluates the only one column value with given function result. +func (col *Column) ReverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) { + return types.ChangeReverseResultByUpperLowerBound(sc, col.RetType, res, rType) +} diff --git a/expression/constant.go b/expression/constant.go index be33d1f17ce41..5104b564072fc 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -448,3 +448,16 @@ func (c *Constant) Vectorized() bool { } return true } + +// SupportReverseEval checks whether the builtinFunc support reverse evaluation. +func (c *Constant) SupportReverseEval() bool { + if c.DeferredExpr != nil { + return c.DeferredExpr.SupportReverseEval() + } + return true +} + +// ReverseEval evaluates the only one column value with given function result. +func (c *Constant) ReverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) { + return c.Value, nil +} diff --git a/expression/errors.go b/expression/errors.go index d0f7d128784fd..5db1acbd0b494 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -29,6 +29,7 @@ var ( ErrOperandColumns = terror.ClassExpression.New(mysql.ErrOperandColumns, mysql.MySQLErrName[mysql.ErrOperandColumns]) ErrCutValueGroupConcat = terror.ClassExpression.New(mysql.ErrCutValueGroupConcat, mysql.MySQLErrName[mysql.ErrCutValueGroupConcat]) ErrFunctionsNoopImpl = terror.ClassExpression.New(mysql.ErrNotSupportedYet, "function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions") + ErrIncorrectType = terror.ClassExpression.New(mysql.ErrIncorrectType, mysql.MySQLErrName[mysql.ErrIncorrectType]) // All the un-exported errors are defined here: errFunctionNotExists = terror.ClassExpression.New(mysql.ErrSpDoesNotExist, mysql.MySQLErrName[mysql.ErrSpDoesNotExist]) @@ -67,6 +68,7 @@ func init() { mysql.ErrUnknownLocale: mysql.ErrUnknownLocale, mysql.ErrBadField: mysql.ErrBadField, mysql.ErrNonUniq: mysql.ErrNonUniq, + mysql.ErrIncorrectType: mysql.ErrIncorrectType, } terror.ErrClassToMySQLCodes[terror.ClassExpression] = expressionMySQLErrCodes } diff --git a/expression/expression.go b/expression/expression.go index f8ab93986b245..2bb31b995c9f5 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -68,11 +68,21 @@ type VecExpr interface { VecEvalJSON(ctx sessionctx.Context, input *chunk.Chunk, result *chunk.Column) error } +// ReverseExpr contains all resersed evaluation methods. +type ReverseExpr interface { + // SupportReverseEval checks whether the builtinFunc support reverse evaluation. + SupportReverseEval() bool + + // ReverseEval evaluates the only one column value with given function result. + ReverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) +} + // Expression represents all scalar expression in SQL. type Expression interface { fmt.Stringer goJSON.Marshaler VecExpr + ReverseExpr // Eval evaluates an expression through a row. Eval(row chunk.Row) (types.Datum, error) diff --git a/expression/integration_test.go b/expression/integration_test.go index c6e1d3f98093a..c29b3b982408a 100755 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -519,6 +519,9 @@ func (s *testIntegrationSuite2) TestMathBuiltin(c *C) { result = tk.MustQuery(`select truncate(d, -1), truncate(d, 1), truncate(d, -2), truncate(d, 2) from t;`) result.Check(testkit.Rows("10 12.3 0 12.34")) + result = tk.MustQuery(`select truncate(json_array(), 1), truncate("cascasc", 1);`) + result.Check(testkit.Rows("0 0")) + // for pow result = tk.MustQuery("SELECT POW('12', 2), POW(1.2e1, '2.0'), POW(12, 2.0);") result.Check(testkit.Rows("144 144 144")) @@ -3902,24 +3905,73 @@ func (s *testIntegrationSuite) TestFuncJSON(c *C) { r := tk.MustQuery(`select json_type(a), json_type(b) from table_json`) r.Check(testkit.Rows("OBJECT OBJECT", "ARRAY ARRAY")) - r = tk.MustQuery(`select json_unquote('hello'), json_unquote('world')`) - r.Check(testkit.Rows("hello world")) - - r = tk.MustQuery(`select - json_quote(''), - json_quote('""'), - json_quote('a'), - json_quote('3'), - json_quote('{"a": "b"}'), - json_quote('{"a": "b"}'), - json_quote('hello,"quoted string",world'), - json_quote('hello,"宽字符",world'), - json_quote('Invalid Json string is OK'), - json_quote('1\u2232\u22322') - `) - r.Check(testkit.Rows( - `"" "\"\"" "a" "3" "{\"a\": \"b\"}" "{\"a\": \"b\"}" "hello,\"quoted string\",world" "hello,\"宽字符\",world" "Invalid Json string\tis OK" "1u2232u22322"`, - )) + tk.MustGetErrCode("select json_quote();", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_quote('abc', 'def');", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_quote(NULL, 'def');", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_quote('abc', NULL);", mysql.ErrWrongParamcountToNativeFct) + + tk.MustGetErrCode("select json_unquote();", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_unquote('abc', 'def');", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_unquote(NULL, 'def');", mysql.ErrWrongParamcountToNativeFct) + tk.MustGetErrCode("select json_unquote('abc', NULL);", mysql.ErrWrongParamcountToNativeFct) + + tk.MustQuery("select json_quote(NULL);").Check(testkit.Rows("")) + tk.MustQuery("select json_unquote(NULL);").Check(testkit.Rows("")) + + tk.MustQuery("select json_quote('abc');").Check(testkit.Rows(`"abc"`)) + tk.MustQuery(`select json_quote(convert('"abc"' using ascii));`).Check(testkit.Rows(`"\"abc\""`)) + tk.MustQuery(`select json_quote(convert('"abc"' using latin1));`).Check(testkit.Rows(`"\"abc\""`)) + tk.MustQuery(`select json_quote(convert('"abc"' using utf8));`).Check(testkit.Rows(`"\"abc\""`)) + tk.MustQuery(`select json_quote(convert('"abc"' using utf8mb4));`).Check(testkit.Rows(`"\"abc\""`)) + + tk.MustQuery("select json_unquote('abc');").Check(testkit.Rows("abc")) + tk.MustQuery(`select json_unquote('"abc"');`).Check(testkit.Rows("abc")) + tk.MustQuery(`select json_unquote(convert('"abc"' using ascii));`).Check(testkit.Rows("abc")) + tk.MustQuery(`select json_unquote(convert('"abc"' using latin1));`).Check(testkit.Rows("abc")) + tk.MustQuery(`select json_unquote(convert('"abc"' using utf8));`).Check(testkit.Rows("abc")) + tk.MustQuery(`select json_unquote(convert('"abc"' using utf8mb4));`).Check(testkit.Rows("abc")) + + tk.MustQuery(`select json_quote('"');`).Check(testkit.Rows(`"\""`)) + tk.MustQuery(`select json_unquote('"');`).Check(testkit.Rows(`"`)) + + tk.MustQuery(`select json_unquote('""');`).Check(testkit.Rows(``)) + tk.MustQuery(`select char_length(json_unquote('""'));`).Check(testkit.Rows(`0`)) + tk.MustQuery(`select json_unquote('"" ');`).Check(testkit.Rows(`"" `)) + tk.MustQuery(`select json_unquote(cast(json_quote('abc') as json));`).Check(testkit.Rows("abc")) + + tk.MustQuery(`select json_unquote(cast('{"abc": "foo"}' as json));`).Check(testkit.Rows(`{"abc": "foo"}`)) + tk.MustQuery(`select json_unquote(json_extract(cast('{"abc": "foo"}' as json), '$.abc'));`).Check(testkit.Rows("foo")) + tk.MustQuery(`select json_unquote('["a", "b", "c"]');`).Check(testkit.Rows(`["a", "b", "c"]`)) + tk.MustQuery(`select json_unquote(cast('["a", "b", "c"]' as json));`).Check(testkit.Rows(`["a", "b", "c"]`)) + tk.MustQuery(`select json_quote(convert(X'e68891' using utf8));`).Check(testkit.Rows(`"我"`)) + tk.MustQuery(`select json_quote(convert(X'e68891' using utf8mb4));`).Check(testkit.Rows(`"我"`)) + tk.MustQuery(`select cast(json_quote(convert(X'e68891' using utf8)) as json);`).Check(testkit.Rows(`"我"`)) + tk.MustQuery(`select json_unquote(convert(X'e68891' using utf8));`).Check(testkit.Rows("我")) + + tk.MustQuery(`select json_quote(json_quote(json_quote('abc')));`).Check(testkit.Rows(`"\"\\\"abc\\\"\""`)) + tk.MustQuery(`select json_unquote(json_unquote(json_unquote(json_quote(json_quote(json_quote('abc'))))));`).Check(testkit.Rows("abc")) + + tk.MustGetErrCode("select json_quote(123)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_quote(-100)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_quote(123.123)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_quote(-100.000)", mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(true);`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(false);`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(cast("{}" as JSON));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(cast("[]" as JSON));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(cast("2015-07-29" as date));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(cast("12:18:29.000000" as time));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_quote(cast("2015-07-29 12:18:29.000000" as datetime));`, mysql.ErrIncorrectType) + + tk.MustGetErrCode("select json_unquote(123)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_unquote(-100)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_unquote(123.123)", mysql.ErrIncorrectType) + tk.MustGetErrCode("select json_unquote(-100.000)", mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_unquote(true);`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_unquote(false);`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_unquote(cast("2015-07-29" as date));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_unquote(cast("12:18:29.000000" as time));`, mysql.ErrIncorrectType) + tk.MustGetErrCode(`select json_unquote(cast("2015-07-29 12:18:29.000000" as datetime));`, mysql.ErrIncorrectType) r = tk.MustQuery(`select json_extract(a, '$.a[1]'), json_extract(b, '$.b') from table_json`) r.Check(testkit.Rows("\"2\" true", " ")) diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 29a5933741b03..a274215f0c88b 100755 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -86,6 +86,21 @@ func (sf *ScalarFunction) Vectorized() bool { return sf.Function.vectorized() && sf.Function.isChildrenVectorized() } +// SupportReverseEval returns if this expression supports reversed evaluation. +func (sf *ScalarFunction) SupportReverseEval() bool { + switch sf.RetType.Tp { + case mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, + mysql.TypeFloat, mysql.TypeDouble, mysql.TypeNewDecimal: + return sf.Function.supportReverseEval() && sf.Function.isChildrenReversed() + } + return false +} + +// ReverseEval evaluates the only one column value with given function result. +func (sf *ScalarFunction) ReverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) { + return sf.Function.reverseEval(sc, res, rType) +} + // GetCtx gets the context of function. func (sf *ScalarFunction) GetCtx() sessionctx.Context { return sf.Function.getCtx() diff --git a/expression/util_test.go b/expression/util_test.go index 13bea0dc9ec8c..4abda426a4414 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -501,6 +501,9 @@ func (m *MockExpr) EvalJSON(ctx sessionctx.Context, row chunk.Row) (val json.Bin } return json.BinaryJSON{}, m.i == nil, m.err } +func (m *MockExpr) ReverseEval(sc *stmtctx.StatementContext, res types.Datum, rType types.RoundingType) (val types.Datum, err error) { + return types.Datum{}, m.err +} func (m *MockExpr) GetType() *types.FieldType { return m.t } func (m *MockExpr) Clone() Expression { return nil } func (m *MockExpr) Equal(ctx sessionctx.Context, e Expression) bool { return false } @@ -513,3 +516,4 @@ func (m *MockExpr) ExplainInfo() string { return " func (m *MockExpr) ExplainNormalizedInfo() string { return "" } func (m *MockExpr) HashCode(sc *stmtctx.StatementContext) []byte { return nil } func (m *MockExpr) Vectorized() bool { return false } +func (m *MockExpr) SupportReverseEval() bool { return false } diff --git a/go.mod b/go.mod index bbd17235fb0d9..a55aec22a7686 100644 --- a/go.mod +++ b/go.mod @@ -36,7 +36,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20191202044712-32be31591b03 github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 - github.com/pingcap/parser v0.0.0-20191203075000-55f02bc42e92 + github.com/pingcap/parser v0.0.0-20191204131342-259c92691fa4 github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0 github.com/pingcap/sysutil v0.0.0-20191126040022-986c5b3ed9a3 github.com/pingcap/tidb-tools v3.0.6-0.20191106033616-90632dda3863+incompatible diff --git a/go.sum b/go.sum index b70bd62ab358a..c9ce17b96fc53 100644 --- a/go.sum +++ b/go.sum @@ -187,8 +187,8 @@ github.com/pingcap/kvproto v0.0.0-20191202044712-32be31591b03/go.mod h1:WWLmULLO github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 h1:AJD9pZYm72vMgPcQDww9rkZ1DnWfl0pXV3BOWlkYIjA= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20191203075000-55f02bc42e92 h1:lmNqIpKSYnSUZL00nTU/bNSKqT2XS3i1jMh/ujIiOaY= -github.com/pingcap/parser v0.0.0-20191203075000-55f02bc42e92/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20191204131342-259c92691fa4 h1:RdccMHB2TG4FexODGlLPSZe/H+MgHJVVptPMJP+9AcA= +github.com/pingcap/parser v0.0.0-20191204131342-259c92691fa4/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0 h1:GIEq+wZfrl2bcJxpuSrEH4H7/nlf5YdmpS+dU9lNIt8= github.com/pingcap/pd v1.1.0-beta.0.20190923032047-5c648dc365e0/go.mod h1:G/6rJpnYwM0LKMec2rI82/5Kg6GaZMvlfB+e6/tvYmI= github.com/pingcap/sysutil v0.0.0-20191126040022-986c5b3ed9a3 h1:HCNif3lukL83gNC2EBAoh2Qbz36+2p0bm0LjgnNfl1s= diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 976234684d381..80392ac99026a 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -23,8 +23,11 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) var ( @@ -392,3 +395,17 @@ func HasAutoIncrementColumn(tbInfo *model.TableInfo) (bool, string) { } return false, "" } + +// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set, +// Otherwise, snapshot schema is returned. +func GetInfoSchema(ctx sessionctx.Context) InfoSchema { + sessVar := ctx.GetSessionVars() + var is InfoSchema + if snap := sessVar.SnapshotInfoschema; snap != nil { + is = snap.(InfoSchema) + logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion())) + } else { + is = sessVar.TxnCtx.InfoSchema.(InfoSchema) + } + return is +} diff --git a/infoschema/tables.go b/infoschema/tables.go index 48e7922eef54b..cd5a43bece6aa 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -2418,7 +2418,7 @@ func (s schemasSorter) Less(i, j int) bool { } func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) { - is := ctx.GetSessionVars().TxnCtx.InfoSchema.(InfoSchema) + is := GetInfoSchema(ctx) dbs := is.AllSchemas() sort.Sort(schemasSorter(dbs)) switch it.meta.Name.O { diff --git a/planner/cascades/testdata/integration_suite_out.json b/planner/cascades/testdata/integration_suite_out.json index 2f6f775b4e28e..e9bcfcc5bdf8f 100644 --- a/planner/cascades/testdata/integration_suite_out.json +++ b/planner/cascades/testdata/integration_suite_out.json @@ -38,10 +38,9 @@ { "SQL": "select a from t where a * 3 + 1 > 9 and a < 5", "Plan": [ - "Projection_9 4.00 root test.t.a", - "└─TableReader_10 4.00 root data:Selection_11", - " └─Selection_11 4.00 cop[tikv] gt(plus(mul(test.t.a, 3), 1), 9)", - " └─TableScan_12 5.00 cop[tikv] table:t, range:[-inf,5), keep order:false, stats:pseudo" + "TableReader_9 4.00 root data:Selection_10", + "└─Selection_10 4.00 cop[tikv] gt(plus(mul(test.t.a, 3), 1), 9)", + " └─TableScan_11 5.00 cop[tikv] table:t, range:[-inf,5), keep order:false, stats:pseudo" ], "Result": [ "3" @@ -70,9 +69,8 @@ { "SQL": "select a from t order by a", "Plan": [ - "Projection_7 10000.00 root test.t.a", - "└─TableReader_8 10000.00 root data:TableScan_9", - " └─TableScan_9 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, stats:pseudo" + "TableReader_7 10000.00 root data:TableScan_8", + "└─TableScan_8 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, stats:pseudo" ], "Result": [ "1", @@ -84,10 +82,9 @@ { "SQL": "select b from t order by b", "Plan": [ - "Projection_7 10000.00 root test.t.b", - "└─Sort_12 10000.00 root test.t.b:asc", - " └─TableReader_10 10000.00 root data:TableScan_11", - " └─TableScan_11 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "Sort_11 10000.00 root test.t.b:asc", + "└─TableReader_9 10000.00 root data:TableScan_10", + " └─TableScan_10 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "11", @@ -140,11 +137,10 @@ { "SQL": "select sum(a) from t", "Plan": [ - "Projection_8 1.00 root Column#3", - "└─HashAgg_13 1.00 root funcs:sum(Column#4)->Column#3", - " └─TableReader_14 1.00 root data:HashAgg_15", - " └─HashAgg_15 1.00 cop[tikv] funcs:sum(test.t.a)->Column#4", - " └─TableScan_11 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "HashAgg_12 1.00 root funcs:sum(Column#4)->Column#3", + "└─TableReader_13 1.00 root data:HashAgg_14", + " └─HashAgg_14 1.00 cop[tikv] funcs:sum(test.t.a)->Column#4", + " └─TableScan_10 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "10" @@ -153,11 +149,10 @@ { "SQL": "select max(a), min(b) from t", "Plan": [ - "Projection_8 1.00 root Column#3, Column#4", - "└─HashAgg_12 1.00 root funcs:max(Column#5)->Column#3, funcs:min(Column#6)->Column#4", - " └─TableReader_13 1.00 root data:HashAgg_14", - " └─HashAgg_14 1.00 cop[tikv] funcs:max(test.t.a)->Column#5, funcs:min(test.t.b)->Column#6", - " └─TableScan_11 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "HashAgg_11 1.00 root funcs:max(Column#5)->Column#3, funcs:min(Column#6)->Column#4", + "└─TableReader_12 1.00 root data:HashAgg_13", + " └─HashAgg_13 1.00 cop[tikv] funcs:max(test.t.a)->Column#5, funcs:min(test.t.b)->Column#6", + " └─TableScan_10 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "4 11" @@ -218,11 +213,10 @@ { "SQL": "select max(a+b) from t", "Plan": [ - "Projection_8 1.00 root Column#3", - "└─HashAgg_13 1.00 root funcs:max(Column#4)->Column#3", - " └─TableReader_14 1.00 root data:HashAgg_15", - " └─HashAgg_15 1.00 cop[tikv] funcs:max(plus(test.t.a, test.t.b))->Column#4", - " └─TableScan_11 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "HashAgg_12 1.00 root funcs:max(Column#4)->Column#3", + "└─TableReader_13 1.00 root data:HashAgg_14", + " └─HashAgg_14 1.00 cop[tikv] funcs:max(plus(test.t.a, test.t.b))->Column#4", + " └─TableScan_10 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "48" @@ -232,12 +226,11 @@ "SQL": "select sum(a) from t group by a, a+b order by a", "Plan": [ "Projection_10 8000.00 root Column#3", - "└─Projection_12 8000.00 root Column#3, test.t.a", - " └─Sort_21 8000.00 root test.t.a:asc", - " └─HashAgg_14 8000.00 root group by:Column#10, Column#11, funcs:sum(Column#8)->Column#3, funcs:firstrow(Column#9)->test.t.a", - " └─Projection_17 10000.00 root cast(test.t.a)->Column#8, test.t.a, test.t.a, plus(test.t.a, test.t.b)->Column#11", - " └─TableReader_15 10000.00 root data:TableScan_16", - " └─TableScan_16 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "└─Sort_20 8000.00 root test.t.a:asc", + " └─HashAgg_13 8000.00 root group by:Column#10, Column#11, funcs:sum(Column#8)->Column#3, funcs:firstrow(Column#9)->test.t.a", + " └─Projection_16 10000.00 root cast(test.t.a)->Column#8, test.t.a, test.t.a, plus(test.t.a, test.t.b)->Column#11", + " └─TableReader_14 10000.00 root data:TableScan_15", + " └─TableScan_15 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "1", @@ -255,9 +248,8 @@ "SQL": "select a from t limit 2", "Plan": [ "Limit_6 2.00 root offset:0, count:2", - "└─Projection_7 2.00 root test.t.a", - " └─TableReader_8 2.00 root data:TableScan_9", - " └─TableScan_9 2.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "└─TableReader_7 2.00 root data:TableScan_8", + " └─TableScan_8 2.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "1", @@ -268,9 +260,8 @@ "SQL": "select a from t limit 1 offset 2", "Plan": [ "Limit_6 1.00 root offset:2, count:1", - "└─Projection_7 3.00 root test.t.a", - " └─TableReader_8 3.00 root data:TableScan_9", - " └─TableScan_9 3.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "└─TableReader_7 3.00 root data:TableScan_8", + " └─TableScan_8 3.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "3" @@ -279,10 +270,9 @@ { "SQL": "select b from t order by b limit 3", "Plan": [ - "Projection_9 3.00 root test.t.b", - "└─TopN_10 3.00 root test.t.b:asc, offset:0, count:3", - " └─TableReader_12 10000.00 root data:TableScan_13", - " └─TableScan_13 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" + "TopN_8 3.00 root test.t.b:asc, offset:0, count:3", + "└─TableReader_10 10000.00 root data:TableScan_11", + " └─TableScan_11 10000.00 cop[tikv] table:t, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "11", @@ -293,10 +283,9 @@ { "SQL": "select a from t order by a limit 1 offset 2", "Plan": [ - "Projection_9 1.00 root test.t.a", - "└─Limit_11 1.00 root offset:2, count:1", - " └─TableReader_14 3.00 root data:TableScan_15", - " └─TableScan_15 3.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, stats:pseudo" + "Limit_9 1.00 root offset:2, count:1", + "└─TableReader_12 3.00 root data:TableScan_13", + " └─TableScan_13 3.00 cop[tikv] table:t, range:[-inf,+inf], keep order:true, stats:pseudo" ], "Result": [ "3" @@ -305,17 +294,14 @@ { "SQL": "select * from ((select a as aa from t t1) union all (select b as aa from t t2)) as t3 order by aa", "Plan": [ - "Projection_15 20000.00 root Column#5", - "└─Sort_26 20000.00 root Column#5:asc", - " └─Union_17 20000.00 root ", - " ├─Projection_18 10000.00 root test.t.a", - " │ └─Projection_19 10000.00 root test.t.a", - " │ └─TableReader_20 10000.00 root data:TableScan_21", - " │ └─TableScan_21 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - " └─Projection_22 10000.00 root test.t.b", - " └─Projection_23 10000.00 root test.t.b", - " └─TableReader_24 10000.00 root data:TableScan_25", - " └─TableScan_25 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + "Sort_23 20000.00 root Column#5:asc", + "└─Union_16 20000.00 root ", + " ├─Projection_17 10000.00 root test.t.a", + " │ └─TableReader_18 10000.00 root data:TableScan_19", + " │ └─TableScan_19 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + " └─Projection_20 10000.00 root test.t.b", + " └─TableReader_21 10000.00 root data:TableScan_22", + " └─TableScan_22 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" ], "Result": [ "1", @@ -336,9 +322,8 @@ { "SQL": "select b from t", "Plan": [ - "Projection_9 10000.00 root test.t.b", - "└─IndexReader_14 10000.00 root index:IndexScan_15", - " └─IndexScan_15 10000.00 cop[tikv] table:t, index:b, range:[NULL,+inf], keep order:false, stats:pseudo" + "IndexReader_11 10000.00 root index:IndexScan_12", + "└─IndexScan_12 10000.00 cop[tikv] table:t, index:b, range:[NULL,+inf], keep order:false, stats:pseudo" ], "Result": [ "2", @@ -350,9 +335,8 @@ "SQL": "select a from t order by b", "Plan": [ "Projection_11 10000.00 root test.t.a", - "└─Projection_13 10000.00 root test.t.a, test.t.b", - " └─IndexReader_16 10000.00 root index:IndexScan_17", - " └─IndexScan_17 10000.00 cop[tikv] table:t, index:b, range:[NULL,+inf], keep order:true, stats:pseudo" + "└─IndexReader_14 10000.00 root index:IndexScan_15", + " └─IndexScan_15 10000.00 cop[tikv] table:t, index:b, range:[NULL,+inf], keep order:true, stats:pseudo" ], "Result": [ "1", @@ -363,9 +347,8 @@ { "SQL": "select c from t", "Plan": [ - "Projection_7 10000.00 root test.t.c", - "└─IndexReader_10 10000.00 root index:IndexScan_11", - " └─IndexScan_11 10000.00 cop[tikv] table:t, index:c, b, range:[NULL,+inf], keep order:false, stats:pseudo" + "IndexReader_9 10000.00 root index:IndexScan_10", + "└─IndexScan_10 10000.00 cop[tikv] table:t, index:c, b, range:[NULL,+inf], keep order:false, stats:pseudo" ], "Result": [ "3", @@ -377,9 +360,8 @@ "SQL": "select a from t order by c", "Plan": [ "Projection_9 10000.00 root test.t.a", - "└─Projection_11 10000.00 root test.t.a, test.t.c", - " └─IndexReader_13 10000.00 root index:IndexScan_14", - " └─IndexScan_14 10000.00 cop[tikv] table:t, index:c, b, range:[NULL,+inf], keep order:true, stats:pseudo" + "└─IndexReader_12 10000.00 root index:IndexScan_13", + " └─IndexScan_13 10000.00 cop[tikv] table:t, index:c, b, range:[NULL,+inf], keep order:true, stats:pseudo" ], "Result": [ "1", diff --git a/planner/cascades/testdata/transformation_rules_suite_in.json b/planner/cascades/testdata/transformation_rules_suite_in.json index f8d7d9c8eebee..0f04fca477a35 100644 --- a/planner/cascades/testdata/transformation_rules_suite_in.json +++ b/planner/cascades/testdata/transformation_rules_suite_in.json @@ -27,7 +27,19 @@ "select b from t order by a limit 2", "select a+b from t order by a limit 1 offset 2", "select c from t order by t.a limit 1", - "select c from t order by t.a + t.b limit 1" + "select c from t order by t.a + t.b limit 1", + "select 1 as a from dual limit 1", + "select a, b, c from t t1 where t1.a in (select t2.a as a from t t2 where t2.b > t1.b order by t1.b limit 1)", + "select a, b, c from t t1 where t1.a in (select a from (select t2.a as a, t1.b as b from t t2 where t2.b > t1.b) x order by b limit 1)", + "select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1" + ] + }, + { + "name": "TestProjectionElimination", + "cases": [ + "select a, b from (select a, b from t) as t2", + "select a+b from (select a, b from t) as t2", + "select a from (select floor(a) as a from t) as t2" ] } ] diff --git a/planner/cascades/testdata/transformation_rules_suite_out.json b/planner/cascades/testdata/transformation_rules_suite_out.json index 53a72766631f5..6bdcc611ea84d 100644 --- a/planner/cascades/testdata/transformation_rules_suite_out.json +++ b/planner/cascades/testdata/transformation_rules_suite_out.json @@ -260,6 +260,130 @@ "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", " TableScan_6 table:t, pk col:test.t.a" ] + }, + { + "SQL": "select 1 as a from dual limit 1", + "Result": [ + "Group#0 Schema:[Column#1]", + " Limit_3 input:[Group#1], offset:0, count:1", + "Group#1 Schema:[Column#1]", + " Projection_2 input:[Group#2], 1->Column#1", + "Group#2 Schema:[]", + " TableDual_1 rowcount:1" + ] + }, + { + "SQL": "select a, b, c from t t1 where t1.a in (select t2.a as a from t t2 where t2.b > t1.b order by t1.b limit 1)", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", + " Projection_9 input:[Group#1], test.t.a, test.t.b, test.t.c", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", + " Apply_8 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " TiKVSingleGather_11 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", + " TableScan_10 table:t1, pk col:test.t.a", + "Group#3 Schema:[test.t.a]", + " Projection_5 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a,test.t.b]", + " TopN_15 input:[Group#6], , offset:0, count:1", + "Group#6 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#7], gt(test.t.b, test.t.b)", + "Group#7 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_13 input:[Group#8], table:t2", + "Group#8 Schema:[test.t.a,test.t.b]", + " TableScan_12 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select a, b, c from t t1 where t1.a in (select a from (select t2.a as a, t1.b as b from t t2 where t2.b > t1.b) x order by b limit 1)", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b,test.t.c]", + " Projection_11 input:[Group#1], test.t.a, test.t.b, test.t.c", + "Group#1 Schema:[test.t.a,test.t.b,test.t.c]", + " Apply_10 input:[Group#2,Group#3], semi join, equal:[eq(test.t.a, test.t.a)]", + "Group#2 Schema:[test.t.a,test.t.b,test.t.c]", + " TiKVSingleGather_13 input:[Group#4], table:t1", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c]", + " TableScan_12 table:t1, pk col:test.t.a", + "Group#3 Schema:[test.t.a]", + " Projection_9 input:[Group#5], test.t.a", + "Group#5 Schema:[test.t.a,Column#25]", + " Projection_6 input:[Group#6], test.t.a, Column#25", + "Group#6 Schema:[test.t.a,Column#25]", + " Projection_5 input:[Group#7], test.t.a, test.t.b", + "Group#7 Schema:[test.t.a,test.t.b]", + " TopN_18 input:[Group#8], test.t.b:asc, offset:0, count:1", + "Group#8 Schema:[test.t.a,test.t.b]", + " Selection_4 input:[Group#9], gt(test.t.b, test.t.b)", + "Group#9 Schema:[test.t.a,test.t.b]", + " TiKVSingleGather_15 input:[Group#10], table:t2", + "Group#10 Schema:[test.t.a,test.t.b]", + " TableScan_14 table:t2, pk col:test.t.a" + ] + }, + { + "SQL": "select a, b from (select @i as a, @i := @i+1 as b from t) t order by a desc limit 1", + "Result": [ + "Group#0 Schema:[Column#13,Column#14]", + " Projection_3 input:[Group#1], Column#13, Column#14", + "Group#1 Schema:[Column#13,Column#14]", + " TopN_21 input:[Group#2], Column#13:desc, offset:0, count:1", + "Group#2 Schema:[Column#13,Column#14]", + " Projection_2 input:[Group#3], getvar(i)->Column#13, setvar(i, cast(plus(cast(getvar(i)), 1)))->Column#14", + "Group#3 Schema:[test.t.a]", + " TiKVSingleGather_7 input:[Group#4], table:t", + " TiKVSingleGather_19 input:[Group#5], table:t, index:e_d_c_str_prefix", + " TiKVSingleGather_17 input:[Group#6], table:t, index:c_d_e_str", + " TiKVSingleGather_15 input:[Group#7], table:t, index:f_g", + " TiKVSingleGather_13 input:[Group#8], table:t, index:g", + " TiKVSingleGather_11 input:[Group#9], table:t, index:f", + " TiKVSingleGather_9 input:[Group#10], table:t, index:c_d_e", + "Group#4 Schema:[test.t.a]", + " TableScan_6 table:t, pk col:test.t.a", + "Group#5 Schema:[test.t.a]", + " IndexScan_18 table:t, index:e_str, d_str, c_str", + "Group#6 Schema:[test.t.a]", + " IndexScan_16 table:t, index:c_str, d_str, e_str", + "Group#7 Schema:[test.t.a]", + " IndexScan_14 table:t, index:f, g", + "Group#8 Schema:[test.t.a]", + " IndexScan_12 table:t, index:g", + "Group#9 Schema:[test.t.a]", + " IndexScan_10 table:t, index:f", + "Group#10 Schema:[test.t.a]", + " IndexScan_8 table:t, index:c, d, e" + ] + } + ] + }, + { + "Name": "TestProjectionElimination", + "Cases": [ + { + "SQL": "select a, b from (select a, b from t) as t2", + "Result": [ + "Group#0 Schema:[test.t.a,test.t.b]", + " TableScan_1 table:t" + ] + }, + { + "SQL": "select a+b from (select a, b from t) as t2", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_3 input:[Group#1], plus(test.t.a, test.t.b)->Column#13", + "Group#1 Schema:[test.t.a,test.t.b]", + " TableScan_1 table:t" + ] + }, + { + "SQL": "select a from (select floor(a) as a from t) as t2", + "Result": [ + "Group#0 Schema:[Column#13]", + " Projection_2 input:[Group#1], floor(test.t.a)->Column#13", + "Group#1 Schema:[test.t.a]", + " TableScan_1 table:t" + ] } ] } diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 695472b86f889..00e2507e808d0 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -60,6 +60,9 @@ var defaultTransformationMap = map[memo.Operand][]Transformation{ memo.OperandLimit: { NewRuleTransformLimitToTopN(), }, + memo.OperandProjection: { + NewRuleEliminateProjection(), + }, memo.OperandTopN: { NewRulePushTopNDownProjection(), }, @@ -595,12 +598,53 @@ func (r *PushSelDownJoin) OnTransform(old *memo.ExprIter) (newExprs []*memo.Grou return []*memo.GroupExpr{newJoinExpr}, true, false, nil } +// EliminateProjection eliminates the projection. +type EliminateProjection struct { + baseRule +} + +// NewRuleEliminateProjection creates a new Transformation EliminateProjection. +// The pattern of this rule is `Projection -> Any`. +func NewRuleEliminateProjection() Transformation { + rule := &EliminateProjection{} + rule.pattern = memo.BuildPattern( + memo.OperandProjection, + memo.EngineTiDBOnly, + memo.NewPattern(memo.OperandAny, memo.EngineTiDBOnly), + ) + return rule +} + +// OnTransform implements Transformation interface. +// This rule tries to eliminate the projection whose output columns are the same with its child. +func (r *EliminateProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { + child := old.Children[0] + if child.Group.Prop.Schema.Len() != old.GetExpr().Group.Prop.Schema.Len() { + return nil, false, false, nil + } + + oldCols := old.GetExpr().Group.Prop.Schema.Columns + for i, col := range child.Group.Prop.Schema.Columns { + if !col.Equal(nil, oldCols[i]) { + return nil, false, false, nil + } + } + + // Promote the children group's expression. + finalGroupExprs := make([]*memo.GroupExpr, 0, child.Group.Equivalents.Len()) + for elem := child.Group.Equivalents.Front(); elem != nil; elem = elem.Next() { + finalGroupExprs = append(finalGroupExprs, elem.Value.(*memo.GroupExpr)) + } + return finalGroupExprs, true, false, nil +} + // PushTopNDownProjection pushes TopN to Projection. type PushTopNDownProjection struct { baseRule } // NewRulePushTopNDownProjection creates a new Transformation PushTopNDownProjection. +// The pattern of this rule is `TopN->Projection->X` to `Projection->TopN->X`. func NewRulePushTopNDownProjection() Transformation { rule := &PushTopNDownProjection{} rule.pattern = memo.BuildPattern( @@ -611,6 +655,17 @@ func NewRulePushTopNDownProjection() Transformation { return rule } +// Match implements Transformation interface. +func (r *PushTopNDownProjection) Match(expr *memo.ExprIter) bool { + proj := expr.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) + for _, expr := range proj.Exprs { + if expression.HasAssignSetVarFunc(expr) { + return false + } + } + return true +} + // OnTransform implements Transformation interface. // This rule tries to pushes the TopN through Projection. func (r *PushTopNDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) { diff --git a/planner/cascades/transformation_rules_test.go b/planner/cascades/transformation_rules_test.go index 1aeeb99b2dc4f..e589b4e4e1c32 100644 --- a/planner/cascades/transformation_rules_test.go +++ b/planner/cascades/transformation_rules_test.go @@ -160,3 +160,21 @@ func (s *testTransformationRuleSuite) TestTopNRules(c *C) { s.testData.GetTestCases(c, &input, &output) testGroupToString(input, output, s, c) } + +func (s *testTransformationRuleSuite) TestProjectionElimination(c *C) { + s.optimizer.ResetTransformationRules(map[memo.Operand][]Transformation{ + memo.OperandProjection: { + NewRuleEliminateProjection(), + }, + }) + defer func() { + s.optimizer.ResetTransformationRules(defaultTransformationMap) + }() + var input []string + var output []struct { + SQL string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + testGroupToString(input, output, s, c) +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 4da7696e17949..4d3b73d69c18c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -300,7 +301,7 @@ func (p *LogicalJoin) constructIndexJoin( innerTask task, ranges []*ranger.Range, keyOff2IdxOff []int, - path *accessPath, + path *util.AccessPath, compareFilters *ColWithCmpFuncManager, ) []PhysicalPlan { joinType := p.JoinType @@ -353,7 +354,7 @@ func (p *LogicalJoin) constructIndexJoin( CompareFilters: compareFilters, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) if path != nil { - join.IdxColLens = path.idxColLens + join.IdxColLens = path.IdxColLens } join.SetSchema(p.schema) return []PhysicalPlan{join} @@ -365,7 +366,7 @@ func (p *LogicalJoin) constructIndexMergeJoin( innerTask task, ranges []*ranger.Range, keyOff2IdxOff []int, - path *accessPath, + path *util.AccessPath, compareFilters *ColWithCmpFuncManager, ) []PhysicalPlan { indexJoins := p.constructIndexJoin(prop, outerIdx, innerTask, ranges, keyOff2IdxOff, path, compareFilters) @@ -442,7 +443,7 @@ func (p *LogicalJoin) constructIndexHashJoin( innerTask task, ranges []*ranger.Range, keyOff2IdxOff []int, - path *accessPath, + path *util.AccessPath, compareFilters *ColWithCmpFuncManager, ) []PhysicalPlan { indexJoins := p.constructIndexJoin(prop, outerIdx, innerTask, ranges, keyOff2IdxOff, path, compareFilters) @@ -519,9 +520,9 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou func (p *LogicalJoin) buildIndexJoinInner2TableScan( prop *property.PhysicalProperty, ds *DataSource, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, us *LogicalUnionScan, avgInnerRowCnt float64) (joins []PhysicalPlan) { - var tblPath *accessPath + var tblPath *util.AccessPath for _, path := range ds.possibleAccessPaths { - if path.isTablePath && path.storeType == kv.TiKV { + if path.IsTablePath && path.StoreType == kv.TiKV { tblPath = path break } @@ -568,7 +569,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( outerIdx int, us *LogicalUnionScan, avgInnerRowCnt float64) (joins []PhysicalPlan) { helper := &indexJoinBuildHelper{join: p} for _, path := range ds.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { continue } emptyRange, err := helper.analyzeLookUpFilters(path, ds, innerJoinKeys) @@ -592,7 +593,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( } } joins = make([]PhysicalPlan, 0, 3) - rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.idxCols, outerJoinKeys) + rangeInfo := helper.buildRangeDecidedByInformation(helper.chosenPath.IdxCols, outerJoinKeys) innerTask := p.constructInnerIndexScanTask(ds, helper.chosenPath, helper.chosenRemained, outerJoinKeys, us, rangeInfo, false, false, avgInnerRowCnt) joins = append(joins, p.constructIndexJoin(prop, outerIdx, innerTask, helper.chosenRanges, keyOff2IdxOff, helper.chosenPath, helper.lastColManager)...) @@ -620,7 +621,7 @@ type indexJoinBuildHelper struct { idxOff2KeyOff []int lastColManager *ColWithCmpFuncManager chosenRanges []*ranger.Range - chosenPath *accessPath + chosenPath *util.AccessPath curPossibleUsedKeys []*expression.Column curNotUsedIndexCols []*expression.Column @@ -720,7 +721,7 @@ func (p *LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physi // constructInnerIndexScanTask is specially used to construct the inner plan for PhysicalIndexJoin. func (p *LogicalJoin) constructInnerIndexScanTask( ds *DataSource, - path *accessPath, + path *util.AccessPath, filterConds []expression.Expression, outerJoinKeys []*expression.Column, us *LogicalUnionScan, @@ -734,9 +735,9 @@ func (p *LogicalJoin) constructInnerIndexScanTask( TableAsName: ds.TableAsName, DBName: ds.DBName, Columns: ds.Columns, - Index: path.index, - IdxCols: path.idxCols, - IdxColLens: path.idxColLens, + Index: path.Index, + IdxCols: path.IdxCols, + IdxColLens: path.IdxColLens, dataSourceSchema: ds.schema, KeepOrder: keepOrder, Ranges: ranger.FullRange(), @@ -752,7 +753,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( tblCols: ds.TblCols, keepOrder: is.KeepOrder, } - if !isCoveringIndex(ds.schema.Columns, path.fullIdxCols, path.fullIdxColLens, is.Table.PKIsHandle) { + if !isCoveringIndex(ds.schema.Columns, path.FullIdxCols, path.FullIdxColLens, is.Table.PKIsHandle) { // On this way, it's double read case. ts := PhysicalTableScan{ Columns: ds.Columns, @@ -768,24 +769,24 @@ func (p *LogicalJoin) constructInnerIndexScanTask( } cop.tablePlan = ts } - is.initSchema(path.index, path.fullIdxCols, cop.tablePlan != nil) - rowSize := is.indexScanRowSize(path.index, ds, true) + is.initSchema(path.Index, path.FullIdxCols, cop.tablePlan != nil) + rowSize := is.indexScanRowSize(path.Index, ds, true) sessVars := ds.ctx.GetSessionVars() cop.cst = rowCount * rowSize * sessVars.ScanFactor - indexConds, tblConds := splitIndexFilterConditions(filterConds, path.fullIdxCols, path.fullIdxColLens, ds.tableInfo) - tmpPath := &accessPath{ - indexFilters: indexConds, - tableFilters: tblConds, - countAfterAccess: rowCount, + indexConds, tblConds := splitIndexFilterConditions(filterConds, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo) + tmpPath := &util.AccessPath{ + IndexFilters: indexConds, + TableFilters: tblConds, + CountAfterAccess: rowCount, } // Assume equal conditions used by index join and other conditions are independent. if len(indexConds) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, indexConds) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, indexConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = selectionFactor } - tmpPath.countAfterIndex = rowCount * selectivity + tmpPath.CountAfterIndex = rowCount * selectivity } selectivity := ds.stats.RowCount / ds.tableStats.RowCount finalStats := ds.stats.ScaleByExpectCnt(selectivity * rowCount) @@ -987,15 +988,15 @@ func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc( return notKeyEqAndIn, nil } -func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, innerPlan *DataSource, innerJoinKeys []*expression.Column) (emptyRange bool, err error) { - if len(path.idxCols) == 0 { +func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath, innerPlan *DataSource, innerJoinKeys []*expression.Column) (emptyRange bool, err error) { + if len(path.IdxCols) == 0 { return false, nil } - accesses := make([]expression.Expression, 0, len(path.idxCols)) - ijHelper.resetContextForIndex(innerJoinKeys, path.idxCols, path.idxColLens) + accesses := make([]expression.Expression, 0, len(path.IdxCols)) + ijHelper.resetContextForIndex(innerJoinKeys, path.IdxCols, path.IdxColLens) notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) var remainedEqAndIn []expression.Expression - notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(path.idxCols, notKeyEqAndIn) + notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(path.IdxCols, notKeyEqAndIn) matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) // If no join key is matched while join keys actually are not empty. We don't choose index join for now. if matchedKeyCnt <= 0 && len(innerJoinKeys) > 0 { @@ -1010,7 +1011,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, inn return false, nil } // If all the index columns are covered by eq/in conditions, we don't need to consider other conditions anymore. - if lastColPos == len(path.idxCols) { + if lastColPos == len(path.IdxCols) { // If there's join key matched index column. Then choose hash join is always a better idea. // e.g. select * from t1, t2 where t2.a=1 and t2.b=1. And t2 has index(a, b). // If we don't have the following check, TiDB will build index join for this case. @@ -1028,10 +1029,10 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, inn ijHelper.updateBestChoice(ranges, path, accesses, remained, nil) return false, nil } - lastPossibleCol := path.idxCols[lastColPos] + lastPossibleCol := path.IdxCols[lastColPos] lastColManager := &ColWithCmpFuncManager{ TargetCol: lastPossibleCol, - colLength: path.idxColLens[lastColPos], + colLength: path.IdxColLens[lastColPos], affectedColSchema: expression.NewSchema(), } lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, innerPlan, lastColManager) @@ -1047,7 +1048,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, inn var ranges, nextColRange []*ranger.Range var err error if len(colAccesses) > 0 { - nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, path.idxColLens[lastColPos]) + nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, path.IdxColLens[lastColPos]) if err != nil { return false, err } @@ -1060,7 +1061,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, inn return true, nil } remained = append(remained, colRemained...) - if path.idxColLens[lastColPos] != types.UnspecifiedLength { + if path.IdxColLens[lastColPos] != types.UnspecifiedLength { remained = append(remained, colAccesses...) } accesses = append(accesses, colAccesses...) @@ -1080,7 +1081,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *accessPath, inn return false, nil } -func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, path *accessPath, accesses, +func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, path *util.AccessPath, accesses, remained []expression.Expression, lastColManager *ColWithCmpFuncManager) { // We choose the index by the number of used columns of the range, the much the better. // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 98fba2158aaa6..dcfe267628535 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" ) @@ -120,9 +121,9 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { DBName: model.NewCIStr("test"), }) joinNode.SetSchema(expression.MergeSchema(dsSchema, outerChildSchema)) - path := &accessPath{ - idxCols: append(make([]*expression.Column, 0, 4), dsSchema.Columns...), - idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, 2, types.UnspecifiedLength}, + path := &util.AccessPath{ + IdxCols: append(make([]*expression.Column, 0, 4), dsSchema.Columns...), + IdxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, 2, types.UnspecifiedLength}, } joinColNames := append(dsNames.Shallow(), outerChildNames...) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e5633b82a1970..2aab18735460c 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" @@ -216,7 +217,7 @@ func (ds *DataSource) tryToGetDualTask() (task, error) { // candidatePath is used to maintain required info for skyline pruning. type candidatePath struct { - path *accessPath + path *util.AccessPath columnSet *intsets.Sparse // columnSet is the set of columns that occurred in the access conditions. isSingleScan bool isMatchProp bool @@ -275,41 +276,41 @@ func compareCandidates(lhs, rhs *candidatePath) int { return 0 } -func (ds *DataSource) getTableCandidate(path *accessPath, prop *property.PhysicalProperty) *candidatePath { +func (ds *DataSource) getTableCandidate(path *util.AccessPath, prop *property.PhysicalProperty) *candidatePath { candidate := &candidatePath{path: path} pkCol := ds.getPKIsHandleCol() if len(prop.Items) == 1 && pkCol != nil { candidate.isMatchProp = prop.Items[0].Col.Equal(nil, pkCol) - if path.storeType == kv.TiFlash { + if path.StoreType == kv.TiFlash { candidate.isMatchProp = candidate.isMatchProp && !prop.Items[0].Desc } } - candidate.columnSet = expression.ExtractColumnSet(path.accessConds) + candidate.columnSet = expression.ExtractColumnSet(path.AccessConds) candidate.isSingleScan = true return candidate } -func (ds *DataSource) getIndexCandidate(path *accessPath, prop *property.PhysicalProperty, isSingleScan bool) *candidatePath { +func (ds *DataSource) getIndexCandidate(path *util.AccessPath, prop *property.PhysicalProperty, isSingleScan bool) *candidatePath { candidate := &candidatePath{path: path} all, _ := prop.AllSameOrder() // When the prop is empty or `all` is false, `isMatchProp` is better to be `false` because // it needs not to keep order for index scan. if !prop.IsEmpty() && all { - for i, col := range path.idxCols { + for i, col := range path.IdxCols { if col.Equal(nil, prop.Items[0].Col) { - candidate.isMatchProp = matchIndicesProp(path.idxCols[i:], path.idxColLens[i:], prop.Items) + candidate.isMatchProp = matchIndicesProp(path.IdxCols[i:], path.IdxColLens[i:], prop.Items) break - } else if i >= path.eqCondCount { + } else if i >= path.EqCondCount { break } } } - candidate.columnSet = expression.ExtractColumnSet(path.accessConds) + candidate.columnSet = expression.ExtractColumnSet(path.AccessConds) candidate.isSingleScan = isSingleScan return candidate } -func (ds *DataSource) getIndexMergeCandidate(path *accessPath) *candidatePath { +func (ds *DataSource) getIndexMergeCandidate(path *util.AccessPath) *candidatePath { candidate := &candidatePath{path: path} return candidate } @@ -319,20 +320,20 @@ func (ds *DataSource) getIndexMergeCandidate(path *accessPath) *candidatePath { func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candidatePath { candidates := make([]*candidatePath, 0, 4) for _, path := range ds.possibleAccessPaths { - if path.partialIndexPaths != nil { + if path.PartialIndexPaths != nil { candidates = append(candidates, ds.getIndexMergeCandidate(path)) continue } // if we already know the range of the scan is empty, just return a TableDual - if len(path.ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { + if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { return []*candidatePath{{path: path}} } var currentCandidate *candidatePath - if path.isTablePath { + if path.IsTablePath { currentCandidate = ds.getTableCandidate(path, prop) } else { - coveredByIdx := isCoveringIndex(ds.schema.Columns, path.fullIdxCols, path.fullIdxColLens, ds.tableInfo.PKIsHandle) - if len(path.accessConds) > 0 || !prop.IsEmpty() || path.forced || coveredByIdx { + coveredByIdx := isCoveringIndex(ds.schema.Columns, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo.PKIsHandle) + if len(path.AccessConds) > 0 || !prop.IsEmpty() || path.Forced || coveredByIdx { // We will use index to generate physical plan if any of the following conditions is satisfied: // 1. This path's access cond is not nil. // 2. We have a non-empty prop to match. @@ -345,7 +346,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida } pruned := false for i := len(candidates) - 1; i >= 0; i-- { - if candidates[i].path.storeType == kv.TiFlash { + if candidates[i].path.StoreType == kv.TiFlash { continue } result := compareCandidates(candidates[i], currentCandidate) @@ -416,7 +417,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err for _, candidate := range candidates { path := candidate.path - if path.partialIndexPaths != nil { + if path.PartialIndexPaths != nil { idxMergeTask, err := ds.convertToIndexMergeScan(prop, candidate) if err != nil { return nil, err @@ -427,18 +428,18 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err continue } // if we already know the range of the scan is empty, just return a TableDual - if len(path.ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { + if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) dual.SetSchema(ds.schema) return &rootTask{ p: dual, }, nil } - if path.isTablePath { - if ds.preferStoreType&preferTiFlash != 0 && path.storeType == kv.TiKV { + if path.IsTablePath { + if ds.preferStoreType&preferTiFlash != 0 && path.StoreType == kv.TiKV { continue } - if ds.preferStoreType&preferTiKV != 0 && path.storeType == kv.TiFlash { + if ds.preferStoreType&preferTiKV != 0 && path.StoreType == kv.TiFlash { continue } tblTask, err := ds.convertToTableScan(prop, candidate) @@ -472,17 +473,17 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c } path := candidate.path var totalCost, totalRowCount float64 - scans := make([]PhysicalPlan, 0, len(path.partialIndexPaths)) + scans := make([]PhysicalPlan, 0, len(path.PartialIndexPaths)) cop := &copTask{ indexPlanFinished: true, tblColHists: ds.TblColHists, } allCovered := true - for _, partPath := range path.partialIndexPaths { + for _, partPath := range path.PartialIndexPaths { var scan PhysicalPlan var partialCost, rowCount float64 var tempCovered bool - if partPath.isTablePath { + if partPath.IsTablePath { scan, partialCost, rowCount, tempCovered = ds.convertToPartialTableScan(prop, partPath) } else { scan, partialCost, rowCount, tempCovered = ds.convertToPartialIndexScan(prop, partPath) @@ -493,8 +494,8 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c allCovered = allCovered && tempCovered } - if !allCovered || len(path.tableFilters) > 0 { - ts, partialCost := ds.buildIndexMergeTableScan(prop, path.tableFilters, totalRowCount) + if !allCovered || len(path.TableFilters) > 0 { + ts, partialCost := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) totalCost += partialCost cop.tablePlan = ts } @@ -504,22 +505,22 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c return task, nil } -func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, path *accessPath) ( +func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, path *util.AccessPath) ( indexPlan PhysicalPlan, partialCost float64, rowCount float64, isCovered bool) { - idx := path.index + idx := path.Index is, partialCost, rowCount := ds.getOriginalPhysicalIndexScan(prop, path, false, false) rowSize := is.indexScanRowSize(idx, ds, false) - isCovered = isCoveringIndex(ds.schema.Columns, path.fullIdxCols, path.fullIdxColLens, ds.tableInfo.PKIsHandle) - indexConds := path.indexFilters + isCovered = isCoveringIndex(ds.schema.Columns, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo.PKIsHandle) + indexConds := path.IndexFilters sessVars := ds.ctx.GetSessionVars() if indexConds != nil { var selectivity float64 partialCost += rowCount * sessVars.CopCPUFactor - if path.countAfterAccess > 0 { - selectivity = path.countAfterIndex / path.countAfterAccess + if path.CountAfterAccess > 0 { + selectivity = path.CountAfterIndex / path.CountAfterAccess } rowCount = is.stats.RowCount * selectivity stats := &property.StatsInfo{RowCount: rowCount} @@ -537,7 +538,7 @@ func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, return indexPlan, partialCost, rowCount, isCovered } -func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *accessPath) ( +func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath) ( tablePlan PhysicalPlan, partialCost float64, rowCount float64, @@ -546,7 +547,7 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, rowSize := ds.TblColHists.GetAvgRowSize(ds.TblCols, false) sessVars := ds.ctx.GetSessionVars() if len(ts.filterCondition) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = selectionFactor @@ -589,7 +590,7 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, } if len(tableFilters) > 0 { partialCost += totalRowCount * sessVars.CopCPUFactor - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, tableFilters) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, tableFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = selectionFactor @@ -749,9 +750,9 @@ func (is *PhysicalIndexScan) initSchema(idx *model.IndexInfo, idxExprCols []*exp is.SetSchema(expression.NewSchema(indexCols...)) } -func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSource, path *accessPath, finalStats *property.StatsInfo) { +func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSource, path *util.AccessPath, finalStats *property.StatsInfo) { // Add filter condition to table plan now. - indexConds, tableConds := path.indexFilters, path.tableFilters + indexConds, tableConds := path.IndexFilters, path.TableFilters tableConds, copTask.rootTaskConds = splitSelCondsWithVirtualColumn(tableConds) @@ -759,8 +760,8 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou if indexConds != nil { copTask.cst += copTask.count() * sessVars.CopCPUFactor var selectivity float64 - if path.countAfterAccess > 0 { - selectivity = path.countAfterIndex / path.countAfterAccess + if path.CountAfterAccess > 0 { + selectivity = path.CountAfterIndex / path.CountAfterAccess } count := is.stats.RowCount * selectivity stats := p.tableStats.ScaleByExpectCnt(count) @@ -905,19 +906,19 @@ func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, return convertedRanges, count, false } -// crossEstimateRowCount estimates row count of table scan using histogram of another column which is in tableFilters +// crossEstimateRowCount estimates row count of table scan using histogram of another column which is in TableFilters // and has high order correlation with handle column. For example, if the query is like: // `select * from tbl where a = 1 order by pk limit 1` // if order of column `a` is strictly correlated with column `pk`, the row count of table scan should be: // `1 + row_count(a < 1 or a is null)` -func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float64, desc bool) (float64, bool, float64) { - if ds.statisticTable.Pseudo || len(path.tableFilters) == 0 { +func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { + if ds.statisticTable.Pseudo || len(path.TableFilters) == 0 { return 0, false, 0 } - col, corr := getMostCorrColFromExprs(path.tableFilters, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) + col, corr := getMostCorrColFromExprs(path.TableFilters, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) // If table scan is not full range scan, we cannot use histogram of other columns for estimation, because // the histogram reflects value distribution in the whole table level. - if col == nil || len(path.accessConds) > 0 { + if col == nil || len(path.AccessConds) > 0 { return 0, false, corr } colInfoID := col.ID @@ -926,7 +927,7 @@ func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float6 if colHist.Correlation < 0 { desc = !desc } - accessConds, remained := ranger.DetachCondsForColumn(ds.ctx, path.tableFilters, col) + accessConds, remained := ranger.DetachCondsForColumn(ds.ctx, path.TableFilters, col) if len(accessConds) == 0 { return 0, false, corr } @@ -945,7 +946,7 @@ func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float6 } convertedRanges, count, isFull := convertRangeFromExpectedCnt(ranges, rangeCounts, expectedCnt, desc) if isFull { - return path.countAfterAccess, true, 0 + return path.CountAfterAccess, true, 0 } var rangeCount float64 if idxExists { @@ -960,7 +961,7 @@ func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float6 if len(remained) > 0 { scanCount = scanCount / selectionFactor } - scanCount = math.Min(scanCount, path.countAfterAccess) + scanCount = math.Min(scanCount, path.CountAfterAccess) return scanCount, true, 0 } @@ -1053,7 +1054,7 @@ func (ts *PhysicalTableScan) addPushedDownSelection(copTask *copTask, stats *pro } } -func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProperty, path *accessPath, isMatchProp bool) (*PhysicalTableScan, float64, float64) { +func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool) (*PhysicalTableScan, float64, float64) { ts := PhysicalTableScan{ Table: ds.tableInfo, Columns: ds.Columns, @@ -1061,10 +1062,10 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper DBName: ds.DBName, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, - Ranges: path.ranges, - AccessCondition: path.accessConds, - filterCondition: path.tableFilters, - StoreType: path.storeType, + Ranges: path.Ranges, + AccessCondition: path.AccessConds, + filterCondition: path.TableFilters, + StoreType: path.StoreType, }.Init(ds.ctx, ds.blockOffset) if ts.StoreType == kv.TiFlash { // Append the AccessCondition to filterCondition because TiFlash only support full range scan for each @@ -1080,7 +1081,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper } } } - rowCount := path.countAfterAccess + rowCount := path.CountAfterAccess if prop.ExpectedCnt < ds.stats.RowCount { count, ok, corr := ds.crossEstimateRowCount(path, prop.ExpectedCnt, isMatchProp && prop.Items[0].Desc) if ok { @@ -1130,18 +1131,18 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper return ts, cost, rowCount } -func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProperty, path *accessPath, isMatchProp bool, isSingleScan bool) (*PhysicalIndexScan, float64, float64) { - idx := path.index +func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool, isSingleScan bool) (*PhysicalIndexScan, float64, float64) { + idx := path.Index is := PhysicalIndexScan{ Table: ds.tableInfo, TableAsName: ds.TableAsName, DBName: ds.DBName, Columns: ds.Columns, Index: idx, - IdxCols: path.idxCols, - IdxColLens: path.idxColLens, - AccessCondition: path.accessConds, - Ranges: path.ranges, + IdxCols: path.IdxCols, + IdxColLens: path.IdxColLens, + AccessCondition: path.AccessConds, + Ranges: path.Ranges, dataSourceSchema: ds.schema, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, @@ -1150,13 +1151,13 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper if statsTbl.Indices[idx.ID] != nil { is.Hist = &statsTbl.Indices[idx.ID].Histogram } - rowCount := path.countAfterAccess - is.initSchema(idx, path.fullIdxCols, !isSingleScan) + rowCount := path.CountAfterAccess + is.initSchema(idx, path.FullIdxCols, !isSingleScan) // Only use expectedCnt when it's smaller than the count we calculated. // e.g. IndexScan(count1)->After Filter(count2). The `ds.stats.RowCount` is count2. count1 is the one we need to calculate // If expectedCnt and count2 are both zero and we go into the below `if` block, the count1 will be set to zero though it's shouldn't be. if (isMatchProp || prop.IsEmpty()) && prop.ExpectedCnt < ds.stats.RowCount { - selectivity := ds.stats.RowCount / path.countAfterAccess + selectivity := ds.stats.RowCount / path.CountAfterAccess rowCount = math.Min(prop.ExpectedCnt/selectivity, rowCount) } is.stats = ds.tableStats.ScaleByExpectCnt(rowCount) diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index bdbc86001c58f..bbcc701a0754a 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" @@ -49,7 +50,7 @@ func (s *testIndexMergeSuite) TearDownSuite(c *C) { c.Assert(s.testdata.GenerateOutputIfNeeded(), IsNil) } -func getIndexMergePathDigest(paths []*accessPath, startIndex int) string { +func getIndexMergePathDigest(paths []*util.AccessPath, startIndex int) string { if len(paths) == startIndex { return "[]" } @@ -60,18 +61,18 @@ func getIndexMergePathDigest(paths []*accessPath, startIndex int) string { } path := paths[i] idxMergeDisgest += "{Idxs:[" - for j := 0; j < len(path.partialIndexPaths); j++ { + for j := 0; j < len(path.PartialIndexPaths); j++ { if j > 0 { idxMergeDisgest += "," } - idxMergeDisgest += path.partialIndexPaths[j].index.Name.L + idxMergeDisgest += path.PartialIndexPaths[j].Index.Name.L } idxMergeDisgest += "],TbFilters:[" - for j := 0; j < len(path.tableFilters); j++ { + for j := 0; j < len(path.TableFilters); j++ { if j > 0 { idxMergeDisgest += "," } - idxMergeDisgest += path.tableFilters[j].String() + idxMergeDisgest += path.TableFilters[j].String() } idxMergeDisgest += "]}" } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 8a2a8f31c11fb..15d8f29567d47 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" @@ -452,7 +453,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { ds.preferStoreType |= preferTiFlash hasTiFlashPath := false for _, path := range ds.possibleAccessPaths { - if path.storeType == kv.TiFlash { + if path.StoreType == kv.TiFlash { hasTiFlashPath = true break } @@ -460,7 +461,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { // TODO: For now, if there is a TiFlash hint for a table, we enforce a TiFlash path. But hint is just a suggestion // for the planner. We can keep it since we need it to debug with PD and TiFlash. In future, this should be removed. if !hasTiFlashPath { - ds.possibleAccessPaths = append(ds.possibleAccessPaths, &accessPath{isTablePath: true, storeType: kv.TiFlash}) + ds.possibleAccessPaths = append(ds.possibleAccessPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash}) } } } @@ -2631,10 +2632,10 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as ds.names = names ds.setPreferredStoreType(b.TableHints()) - // Init fullIdxCols, fullIdxColLens for accessPaths. + // Init FullIdxCols, FullIdxColLens for accessPaths. for _, path := range ds.possibleAccessPaths { - if !path.isTablePath { - path.fullIdxCols, path.fullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.index) + if !path.IsTablePath { + path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.Index) } } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 79a1dd565c86f..8c8f139ae323b 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1349,10 +1349,10 @@ func byItemsToProperty(byItems []*ByItems) *property.PhysicalProperty { func pathsName(paths []*candidatePath) string { var names []string for _, path := range paths { - if path.path.isTablePath { + if path.path.IsTablePath { names = append(names, "PRIMARY_KEY") } else { - names = append(names, path.path.index.Name.O) + names = append(names, path.path.Index.Name.O) } } return strings.Join(names, ",") diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 265713839186f..5bfa6614b4638 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -408,7 +408,7 @@ type DataSource struct { tableStats *property.StatsInfo // possibleAccessPaths stores all the possible access path for physical plan, including table scan. - possibleAccessPaths []*accessPath + possibleAccessPaths []*util.AccessPath // The data source may be a partition, rather than a real table. isPartition bool @@ -486,37 +486,10 @@ func (p *LogicalIndexScan) MatchIndexProp(prop *property.PhysicalProperty) (matc return false } -// accessPath indicates the way we access a table: by using single index, or by using multiple indexes, -// or just by using table scan. -type accessPath struct { - index *model.IndexInfo - fullIdxCols []*expression.Column - fullIdxColLens []int - idxCols []*expression.Column - idxColLens []int - ranges []*ranger.Range - // countAfterAccess is the row count after we apply range seek and before we use other filter to filter data. - countAfterAccess float64 - // countAfterIndex is the row count after we apply filters on index and before we apply the table filters. - countAfterIndex float64 - accessConds []expression.Expression - eqCondCount int - indexFilters []expression.Expression - tableFilters []expression.Expression - // isTablePath indicates whether this path is table path. - isTablePath bool - storeType kv.StoreType - // forced means this path is generated by `use/force index()`. - forced bool - // partialIndexPaths store all index access paths. - // If there are extra filters, store them in tableFilters. - partialIndexPaths []*accessPath -} - // getTablePath finds the TablePath from a group of accessPaths. -func getTablePath(paths []*accessPath) *accessPath { +func getTablePath(paths []*util.AccessPath) *util.AccessPath { for _, path := range paths { - if path.isTablePath { + if path.IsTablePath { return path } } @@ -532,11 +505,11 @@ func (ds *DataSource) buildTableGather() LogicalPlan { return sg } -func (ds *DataSource) buildIndexGather(path *accessPath) LogicalPlan { +func (ds *DataSource) buildIndexGather(path *util.AccessPath) LogicalPlan { is := LogicalIndexScan{ Source: ds, IsDoubleRead: false, - Index: path.index, + Index: path.Index, }.Init(ds.ctx, ds.blockOffset) is.Columns = make([]*model.ColumnInfo, len(ds.Columns)) @@ -546,7 +519,7 @@ func (ds *DataSource) buildIndexGather(path *accessPath) LogicalPlan { sg := TiKVSingleGather{ Source: ds, IsIndexGather: true, - Index: path.index, + Index: path.Index, }.Init(ds.ctx, ds.blockOffset) sg.SetSchema(ds.Schema()) sg.SetChildren(is) @@ -558,10 +531,10 @@ func (ds *DataSource) Convert2Gathers() (gathers []LogicalPlan) { tg := ds.buildTableGather() gathers = append(gathers, tg) for _, path := range ds.possibleAccessPaths { - if !path.isTablePath { - path.fullIdxCols, path.fullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.index) + if !path.IsTablePath { + path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.Index) // If index columns can cover all of the needed columns, we can use a IndexGather + IndexScan. - if isCoveringIndex(ds.schema.Columns, path.fullIdxCols, path.fullIdxColLens, ds.tableInfo.PKIsHandle) { + if isCoveringIndex(ds.schema.Columns, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo.PKIsHandle) { gathers = append(gathers, ds.buildIndexGather(path)) } // TODO: If index columns can not cover the schema, use IndexLookUpGather. @@ -570,14 +543,14 @@ func (ds *DataSource) Convert2Gathers() (gathers []LogicalPlan) { return gathers } -// deriveTablePathStats will fulfill the information that the accessPath need. +// deriveTablePathStats will fulfill the information that the AccessPath need. // And it will check whether the primary key is covered only by point query. // isIm indicates whether this function is called to generate the partial path for IndexMerge. -func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression.Expression, isIm bool) (bool, error) { +func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expression.Expression, isIm bool) (bool, error) { var err error sc := ds.ctx.GetSessionVars().StmtCtx - path.countAfterAccess = float64(ds.statisticTable.Count) - path.tableFilters = conds + path.CountAfterAccess = float64(ds.statisticTable.Count) + path.TableFilters = conds var pkCol *expression.Column columnLen := len(ds.schema.Columns) isUnsigned := false @@ -590,20 +563,20 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression. pkCol = ds.schema.Columns[columnLen-1] } if pkCol == nil { - path.ranges = ranger.FullIntRange(isUnsigned) + path.Ranges = ranger.FullIntRange(isUnsigned) return false, nil } - path.ranges = ranger.FullIntRange(isUnsigned) + path.Ranges = ranger.FullIntRange(isUnsigned) if len(conds) == 0 { return false, nil } - path.accessConds, path.tableFilters = ranger.DetachCondsForColumn(ds.ctx, conds, pkCol) + path.AccessConds, path.TableFilters = ranger.DetachCondsForColumn(ds.ctx, conds, pkCol) // If there's no access cond, we try to find that whether there's expression containing correlated column that // can be used to access data. corColInAccessConds := false - if len(path.accessConds) == 0 { - for i, filter := range path.tableFilters { + if len(path.AccessConds) == 0 { + for i, filter := range path.TableFilters { eqFunc, ok := filter.(*expression.ScalarFunction) if !ok || eqFunc.FuncName.L != ast.EQ { continue @@ -612,8 +585,8 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression. if lOk && lCol.Equal(ds.ctx, pkCol) { _, rOk := eqFunc.GetArgs()[1].(*expression.CorrelatedColumn) if rOk { - path.accessConds = append(path.accessConds, filter) - path.tableFilters = append(path.tableFilters[:i], path.tableFilters[i+1:]...) + path.AccessConds = append(path.AccessConds, filter) + path.TableFilters = append(path.TableFilters[:i], path.TableFilters[i+1:]...) corColInAccessConds = true break } @@ -622,8 +595,8 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression. if rOk && rCol.Equal(ds.ctx, pkCol) { _, lOk := eqFunc.GetArgs()[0].(*expression.CorrelatedColumn) if lOk { - path.accessConds = append(path.accessConds, filter) - path.tableFilters = append(path.tableFilters[:i], path.tableFilters[i+1:]...) + path.AccessConds = append(path.AccessConds, filter) + path.TableFilters = append(path.TableFilters[:i], path.TableFilters[i+1:]...) corColInAccessConds = true break } @@ -631,22 +604,22 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression. } } if corColInAccessConds { - path.countAfterAccess = 1 + path.CountAfterAccess = 1 return true, nil } - path.ranges, err = ranger.BuildTableRange(path.accessConds, sc, pkCol.RetType) + path.Ranges, err = ranger.BuildTableRange(path.AccessConds, sc, pkCol.RetType) if err != nil { return false, err } - path.countAfterAccess, err = ds.statisticTable.GetRowCountByIntColumnRanges(sc, pkCol.ID, path.ranges) - // If the `countAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. + path.CountAfterAccess, err = ds.statisticTable.GetRowCountByIntColumnRanges(sc, pkCol.ID, path.Ranges) + // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. - if path.countAfterAccess < ds.stats.RowCount && !isIm { - path.countAfterAccess = math.Min(ds.stats.RowCount/selectionFactor, float64(ds.statisticTable.Count)) + if path.CountAfterAccess < ds.stats.RowCount && !isIm { + path.CountAfterAccess = math.Min(ds.stats.RowCount/selectionFactor, float64(ds.statisticTable.Count)) } // Check whether the primary key is covered by point query. noIntervalRange := true - for _, ran := range path.ranges { + for _, ran := range path.Ranges { if !ran.IsPoint(sc) { noIntervalRange = false break @@ -655,90 +628,95 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath, conds []expression. return noIntervalRange, err } -// deriveIndexPathStats will fulfill the information that the accessPath need. -// And it will check whether this index is full matched by point query. We will use this check to -// determine whether we remove other paths or not. -// conds is the conditions used to generate the DetachRangeResult for path. -// isIm indicates whether this function is called to generate the partial path for IndexMerge. -func (ds *DataSource) deriveIndexPathStats(path *accessPath, conds []expression.Expression, isIm bool) (bool, error) { +func (ds *DataSource) fillIndexPath(path *util.AccessPath, conds []expression.Expression) error { sc := ds.ctx.GetSessionVars().StmtCtx - path.ranges = ranger.FullRange() - path.countAfterAccess = float64(ds.statisticTable.Count) - path.idxCols, path.idxColLens = expression.IndexInfo2PrefixCols(ds.Columns, ds.schema.Columns, path.index) - path.fullIdxCols, path.fullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.index) - if !path.index.Unique && !path.index.Primary && len(path.index.Columns) == len(path.idxCols) { + path.Ranges = ranger.FullRange() + path.CountAfterAccess = float64(ds.statisticTable.Count) + path.IdxCols, path.IdxColLens = expression.IndexInfo2PrefixCols(ds.Columns, ds.schema.Columns, path.Index) + path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.Index) + if !path.Index.Unique && !path.Index.Primary && len(path.Index.Columns) == len(path.IdxCols) { handleCol := ds.getPKIsHandleCol() if handleCol != nil && !mysql.HasUnsignedFlag(handleCol.RetType.Flag) { - path.idxCols = append(path.idxCols, handleCol) - path.idxColLens = append(path.idxColLens, types.UnspecifiedLength) + path.IdxCols = append(path.IdxCols, handleCol) + path.IdxColLens = append(path.IdxColLens, types.UnspecifiedLength) } } - eqOrInCount := 0 - if len(path.idxCols) != 0 { - res, err := ranger.DetachCondAndBuildRangeForIndex(ds.ctx, conds, path.idxCols, path.idxColLens) + if len(path.IdxCols) != 0 { + res, err := ranger.DetachCondAndBuildRangeForIndex(ds.ctx, conds, path.IdxCols, path.IdxColLens) if err != nil { - return false, err - } - path.ranges = res.Ranges - path.accessConds = res.AccessConds - path.tableFilters = res.RemainedConds - path.eqCondCount = res.EqCondCount - eqOrInCount = res.EqOrInCount - path.countAfterAccess, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(sc, path.index.ID, path.ranges) + return err + } + path.Ranges = res.Ranges + path.AccessConds = res.AccessConds + path.TableFilters = res.RemainedConds + path.EqCondCount = res.EqCondCount + path.EqOrInCondCount = res.EqOrInCount + path.IsDNFCond = res.IsDNFCond + path.CountAfterAccess, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(sc, path.Index.ID, path.Ranges) if err != nil { - return false, err + return err } } else { - path.tableFilters = conds + path.TableFilters = conds } - if eqOrInCount == len(path.accessConds) { - accesses, remained := path.splitCorColAccessCondFromFilters(eqOrInCount) - path.accessConds = append(path.accessConds, accesses...) - path.tableFilters = remained + return nil +} + +// deriveIndexPathStats will fulfill the information that the AccessPath need. +// And it will check whether this index is full matched by point query. We will use this check to +// determine whether we remove other paths or not. +// conds is the conditions used to generate the DetachRangeResult for path. +// isIm indicates whether this function is called to generate the partial path for IndexMerge. +func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, conds []expression.Expression, isIm bool) bool { + sc := ds.ctx.GetSessionVars().StmtCtx + if path.EqOrInCondCount == len(path.AccessConds) { + accesses, remained := path.SplitCorColAccessCondFromFilters(path.EqOrInCondCount) + path.AccessConds = append(path.AccessConds, accesses...) + path.TableFilters = remained if len(accesses) > 0 && ds.statisticTable.Pseudo { - path.countAfterAccess = ds.statisticTable.PseudoAvgCountPerValue() + path.CountAfterAccess = ds.statisticTable.PseudoAvgCountPerValue() } else { - selectivity := path.countAfterAccess / float64(ds.statisticTable.Count) + selectivity := path.CountAfterAccess / float64(ds.statisticTable.Count) for i := range accesses { - col := path.idxCols[eqOrInCount+i] + col := path.IdxCols[path.EqOrInCondCount+i] ndv := ds.getColumnNDV(col.ID) ndv *= selectivity if ndv < 1 { ndv = 1.0 } - path.countAfterAccess = path.countAfterAccess / ndv + path.CountAfterAccess = path.CountAfterAccess / ndv } } } - path.indexFilters, path.tableFilters = splitIndexFilterConditions(path.tableFilters, path.fullIdxCols, path.fullIdxColLens, ds.tableInfo) - // If the `countAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. + path.IndexFilters, path.TableFilters = splitIndexFilterConditions(path.TableFilters, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo) + // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. - if path.countAfterAccess < ds.stats.RowCount && !isIm { - path.countAfterAccess = math.Min(ds.stats.RowCount/selectionFactor, float64(ds.statisticTable.Count)) + if path.CountAfterAccess < ds.stats.RowCount && !isIm { + path.CountAfterAccess = math.Min(ds.stats.RowCount/selectionFactor, float64(ds.statisticTable.Count)) } - if path.indexFilters != nil { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, path.indexFilters) + if path.IndexFilters != nil { + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, path.IndexFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = selectionFactor } if isIm { - path.countAfterIndex = path.countAfterAccess * selectivity + path.CountAfterIndex = path.CountAfterAccess * selectivity } else { - path.countAfterIndex = math.Max(path.countAfterAccess*selectivity, ds.stats.RowCount) + path.CountAfterIndex = math.Max(path.CountAfterAccess*selectivity, ds.stats.RowCount) } } // Check whether there's only point query. noIntervalRanges := true haveNullVal := false - for _, ran := range path.ranges { + for _, ran := range path.Ranges { // Not point or the not full matched. - if !ran.IsPoint(sc) || len(ran.HighVal) != len(path.index.Columns) { + if !ran.IsPoint(sc) || len(ran.HighVal) != len(path.Index.Columns) { noIntervalRanges = false break } // Check whether there's null value. - for i := 0; i < len(path.index.Columns); i++ { + for i := 0; i < len(path.Index.Columns); i++ { if ran.HighVal[i].IsNull() { haveNullVal = true break @@ -748,70 +726,7 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath, conds []expression. break } } - return noIntervalRanges && !haveNullVal, nil -} - -func (path *accessPath) splitCorColAccessCondFromFilters(eqOrInCount int) (access, remained []expression.Expression) { - access = make([]expression.Expression, len(path.idxCols)-eqOrInCount) - used := make([]bool, len(path.tableFilters)) - for i := eqOrInCount; i < len(path.idxCols); i++ { - matched := false - for j, filter := range path.tableFilters { - if used[j] || !isColEqCorColOrConstant(filter, path.idxCols[i]) { - continue - } - matched = true - access[i-eqOrInCount] = filter - if path.idxColLens[i] == types.UnspecifiedLength { - used[j] = true - } - break - } - if !matched { - access = access[:i-eqOrInCount] - break - } - } - for i, ok := range used { - if !ok { - remained = append(remained, path.tableFilters[i]) - } - } - return access, remained -} - -// getEqOrInColOffset checks if the expression is a eq function that one side is constant or correlated column -// and another is column. -func isColEqCorColOrConstant(filter expression.Expression, col *expression.Column) bool { - f, ok := filter.(*expression.ScalarFunction) - if !ok || f.FuncName.L != ast.EQ { - return false - } - if c, ok := f.GetArgs()[0].(*expression.Column); ok { - if _, ok := f.GetArgs()[1].(*expression.Constant); ok { - if col.Equal(nil, c) { - return true - } - } - if _, ok := f.GetArgs()[1].(*expression.CorrelatedColumn); ok { - if col.Equal(nil, c) { - return true - } - } - } - if c, ok := f.GetArgs()[1].(*expression.Column); ok { - if _, ok := f.GetArgs()[0].(*expression.Constant); ok { - if col.Equal(nil, c) { - return true - } - } - if _, ok := f.GetArgs()[0].(*expression.CorrelatedColumn); ok { - if col.Equal(nil, c) { - return true - } - } - } - return false + return noIntervalRanges && !haveNullVal } func getPKIsHandleColFromSchema(cols []*model.ColumnInfo, schema *expression.Schema, pkIsHandle bool) *expression.Column { diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index aa225a3f0b732..c083566121eef 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" @@ -183,14 +184,14 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { c.Assert(err, IsNil, comment) trueFilters = append(trueFilters, trueFilter) } - path := accessPath{ - eqCondCount: 0, - tableFilters: trueFilters, - idxCols: expression.FindPrefixOfIndex(totalSchema.Columns, tt.idxColIDs), - idxColLens: tt.idxColLens, + path := util.AccessPath{ + EqCondCount: 0, + TableFilters: trueFilters, + IdxCols: expression.FindPrefixOfIndex(totalSchema.Columns, tt.idxColIDs), + IdxColLens: tt.idxColLens, } - access, remained := path.splitCorColAccessCondFromFilters(path.eqCondCount) + access, remained := path.SplitCorColAccessCondFromFilters(path.EqCondCount) c.Assert(fmt.Sprintf("%s", access), Equals, tt.access, comment) c.Assert(fmt.Sprintf("%s", remained), Equals, tt.remained, comment) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 7e1bbc5bbffa4..650361cd67ffe 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -584,14 +585,14 @@ func (b *PlanBuilder) detectSelectWindow(sel *ast.SelectStmt) bool { return false } -func getPathByIndexName(paths []*accessPath, idxName model.CIStr, tblInfo *model.TableInfo) *accessPath { - var tablePath *accessPath +func getPathByIndexName(paths []*util.AccessPath, idxName model.CIStr, tblInfo *model.TableInfo) *util.AccessPath { + var tablePath *util.AccessPath for _, path := range paths { - if path.isTablePath { + if path.IsTablePath { tablePath = path continue } - if path.index.Name.L == idxName.L { + if path.Index.Name.L == idxName.L { return path } } @@ -605,26 +606,26 @@ func isPrimaryIndex(indexName model.CIStr) bool { return indexName.L == "primary" } -func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName model.CIStr) ([]*accessPath, error) { +func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName model.CIStr) ([]*util.AccessPath, error) { tblInfo := tbl.Meta() - publicPaths := make([]*accessPath, 0, len(tblInfo.Indices)+2) + publicPaths := make([]*util.AccessPath, 0, len(tblInfo.Indices)+2) tp := kv.TiKV if tbl.Type().IsClusterTable() { tp = kv.TiDB } - publicPaths = append(publicPaths, &accessPath{isTablePath: true, storeType: tp}) + publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: tp}) if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { - publicPaths = append(publicPaths, &accessPath{isTablePath: true, storeType: kv.TiFlash}) + publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash}) } for _, index := range tblInfo.Indices { if index.State == model.StatePublic { - publicPaths = append(publicPaths, &accessPath{index: index}) + publicPaths = append(publicPaths, &util.AccessPath{Index: index}) } } hasScanHint, hasUseOrForce := false, false - available := make([]*accessPath, 0, len(publicPaths)) - ignored := make([]*accessPath, 0, len(publicPaths)) + available := make([]*util.AccessPath, 0, len(publicPaths)) + ignored := make([]*util.AccessPath, 0, len(publicPaths)) // Extract comment-style index hint like /*+ INDEX(t, idx1, idx2) */. indexHintsLen := len(indexHints) @@ -649,7 +650,7 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta if hint.IndexNames == nil && hint.HintType != ast.HintIgnore { if path := getTablePath(publicPaths); path != nil { hasUseOrForce = true - path.forced = true + path.Forced = true available = append(available, path) } } @@ -672,7 +673,7 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta // Currently we don't distinguish between "FORCE" and "USE" because // our cost estimation is not reliable. hasUseOrForce = true - path.forced = true + path.Forced = true available = append(available, path) } } @@ -686,25 +687,25 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta // If we have got "FORCE" or "USE" index hint but got no available index, // we have to use table scan. if len(available) == 0 { - available = append(available, &accessPath{isTablePath: true}) + available = append(available, &util.AccessPath{IsTablePath: true}) } return available, nil } -func (b *PlanBuilder) filterPathByIsolationRead(paths []*accessPath) ([]*accessPath, error) { +func (b *PlanBuilder) filterPathByIsolationRead(paths []*util.AccessPath) ([]*util.AccessPath, error) { // TODO: filter paths with isolation read locations. isolationReadEngines := b.ctx.GetSessionVars().GetIsolationReadEngines() availableEngine := map[kv.StoreType]struct{}{} var availableEngineStr string for i := len(paths) - 1; i >= 0; i-- { - if _, ok := availableEngine[paths[i].storeType]; !ok { - availableEngine[paths[i].storeType] = struct{}{} + if _, ok := availableEngine[paths[i].StoreType]; !ok { + availableEngine[paths[i].StoreType] = struct{}{} if availableEngineStr != "" { availableEngineStr += ", " } - availableEngineStr += paths[i].storeType.Name() + availableEngineStr += paths[i].StoreType.Name() } - if _, ok := isolationReadEngines[paths[i].storeType]; !ok { + if _, ok := isolationReadEngines[paths[i].StoreType]; !ok { paths = append(paths[:i], paths[i+1:]...) } } @@ -717,13 +718,13 @@ func (b *PlanBuilder) filterPathByIsolationRead(paths []*accessPath) ([]*accessP return paths, err } -func removeIgnoredPaths(paths, ignoredPaths []*accessPath, tblInfo *model.TableInfo) []*accessPath { +func removeIgnoredPaths(paths, ignoredPaths []*util.AccessPath, tblInfo *model.TableInfo) []*util.AccessPath { if len(ignoredPaths) == 0 { return paths } - remainedPaths := make([]*accessPath, 0, len(paths)) + remainedPaths := make([]*util.AccessPath, 0, len(paths)) for _, path := range paths { - if path.isTablePath || getPathByIndexName(ignoredPaths, path.index.Name, tblInfo) == nil { + if path.IsTablePath || getPathByIndexName(ignoredPaths, path.Index.Name, tblInfo) == nil { remainedPaths = append(remainedPaths, path) } } diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 3d5c35949696d..0908ff88dde69 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" ) @@ -71,9 +72,9 @@ func (s *testPlanBuilderSuite) TestGetPathByIndexName(c *C) { PKIsHandle: true, } - accessPath := []*accessPath{ - {isTablePath: true}, - {index: &model.IndexInfo{Name: model.NewCIStr("idx")}}, + accessPath := []*util.AccessPath{ + {IsTablePath: true}, + {Index: &model.IndexInfo{Name: model.NewCIStr("idx")}}, } path := getPathByIndexName(accessPath, model.NewCIStr("idx"), tblInfo) diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index ebf523d572484..b45e761785b4b 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -21,7 +21,7 @@ func (ds *DataSource) preparePossibleProperties() [][]*expression.Column { result := make([][]*expression.Column, 0, len(ds.possibleAccessPaths)) for _, path := range ds.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { col := ds.getPKIsHandleCol() if col != nil { result = append(result, []*expression.Column{col}) @@ -29,14 +29,14 @@ func (ds *DataSource) preparePossibleProperties() [][]*expression.Column { continue } - if len(path.idxCols) == 0 { + if len(path.IdxCols) == 0 { continue } - result = append(result, make([]*expression.Column, len(path.idxCols))) - copy(result[len(result)-1], path.idxCols) - for i := 0; i < path.eqCondCount && i+1 < len(path.idxCols); i++ { - result = append(result, make([]*expression.Column, len(path.idxCols)-i-1)) - copy(result[len(result)-1], path.idxCols[i+1:]) + result = append(result, make([]*expression.Column, len(path.IdxCols))) + copy(result[len(result)-1], path.IdxCols) + for i := 0; i < path.EqCondCount && i+1 < len(path.IdxCols); i++ { + result = append(result, make([]*expression.Column, len(path.IdxCols)-i-1)) + copy(result[len(result)-1], path.IdxCols[i+1:]) } } return result diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 2e2a43f39e2c2..e875b85ee85a5 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -227,10 +227,10 @@ func checkIndexCanBeKey(idx *model.IndexInfo, columns []*model.ColumnInfo, schem func (ds *DataSource) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema) { selfSchema.Keys = nil for _, path := range ds.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { continue } - if newKey := checkIndexCanBeKey(path.index, ds.Columns, selfSchema); newKey != nil { + if newKey := checkIndexCanBeKey(path.Index, ds.Columns, selfSchema); newKey != nil { selfSchema.Keys = append(selfSchema.Keys, newKey) } } @@ -253,10 +253,10 @@ func (ts *LogicalTableScan) BuildKeyInfo(selfSchema *expression.Schema, childSch func (is *LogicalIndexScan) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema) { selfSchema.Keys = nil for _, path := range is.Source.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { continue } - if newKey := checkIndexCanBeKey(path.index, is.Columns, selfSchema); newKey != nil { + if newKey := checkIndexCanBeKey(path.Index, is.Columns, selfSchema); newKey != nil { selfSchema.Keys = append(selfSchema.Keys, newKey) } } diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index c27be9823d3da..6398c19be5939 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -126,14 +126,14 @@ func (o *outerJoinEliminator) isInnerJoinKeysContainIndex(innerPlan LogicalPlan, return false, nil } for _, path := range ds.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { continue } - if !path.index.Unique { + if !path.Index.Unique { continue } joinKeysContainIndex := true - for _, idxCol := range path.idxCols { + for _, idxCol := range path.IdxCols { if !joinKeys.Contains(idxCol) { joinKeysContainIndex = false break diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index 365144ca43ab8..09e586c0a28c9 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -49,8 +50,8 @@ func (a *maxMinEliminator) composeAggsByInnerJoin(aggs []*LogicalAggregation) (p return } -// checkColCanUseIndex checks whether there is an accessPath satisfy the conditions: -// 1. all of the selection's condition can be pushed down as accessConds of the path. +// checkColCanUseIndex checks whether there is an AccessPath satisfy the conditions: +// 1. all of the selection's condition can be pushed down as AccessConds of the path. // 2. the path can keep order for `col` after pushing down the conditions. func (a *maxMinEliminator) checkColCanUseIndex(plan LogicalPlan, col *expression.Column, conditions []expression.Expression) bool { switch p := plan.(type) { @@ -58,9 +59,9 @@ func (a *maxMinEliminator) checkColCanUseIndex(plan LogicalPlan, col *expression conditions = append(conditions, p.Conditions...) return a.checkColCanUseIndex(p.children[0], col, conditions) case *DataSource: - // Check whether there is an accessPath can use index for col. + // Check whether there is an AccessPath can use index for col. for _, path := range p.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { // Since table path can contain accessConds of at most one column, // we only need to check if all of the conditions can be pushed down as accessConds // and `col` is the handle column. @@ -73,13 +74,13 @@ func (a *maxMinEliminator) checkColCanUseIndex(plan LogicalPlan, col *expression } else { // For index paths, we have to check: // 1. whether all of the conditions can be pushed down as accessConds. - // 2. whether the accessPath can satisfy the order property of `col` with these accessConds. - result, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, conditions, path.fullIdxCols, path.fullIdxColLens) + // 2. whether the AccessPath can satisfy the order property of `col` with these accessConds. + result, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, conditions, path.FullIdxCols, path.FullIdxColLens) if err != nil || len(result.RemainedConds) != 0 { continue } for i := 0; i <= result.EqCondCount; i++ { - if i < len(path.fullIdxCols) && col.Equal(nil, path.fullIdxCols[i]) { + if i < len(path.FullIdxCols) && col.Equal(nil, path.FullIdxCols[i]) { return true } } @@ -109,7 +110,7 @@ func (a *maxMinEliminator) cloneSubPlans(plan LogicalPlan) LogicalPlan { newDs.schema = p.schema.Clone() newDs.Columns = make([]*model.ColumnInfo, len(p.Columns)) copy(newDs.Columns, p.Columns) - newAccessPaths := make([]*accessPath, 0, len(p.possibleAccessPaths)) + newAccessPaths := make([]*util.AccessPath, 0, len(p.possibleAccessPaths)) for _, path := range p.possibleAccessPaths { newPath := *path newAccessPaths = append(newAccessPaths, &newPath) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index ae439cbc0153f..0e7aa9500ceae 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -164,7 +165,7 @@ func (s *partitionProcessor) prune(ds *DataSource) (LogicalPlan, error) { newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.SCtx(), plancodec.TypeTableScan, &newDataSource, ds.blockOffset) newDataSource.isPartition = true newDataSource.physicalTableID = pi.Definitions[i].ID - newDataSource.possibleAccessPaths = make([]*accessPath, len(ds.possibleAccessPaths)) + newDataSource.possibleAccessPaths = make([]*util.AccessPath, len(ds.possibleAccessPaths)) for i := range ds.possibleAccessPaths { newPath := *ds.possibleAccessPaths[i] newDataSource.possibleAccessPaths[i] = &newPath diff --git a/planner/core/stats.go b/planner/core/stats.go index 0a6bf02c1b8a0..0e9aa4cd432f8 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" @@ -137,24 +138,29 @@ func (ds *DataSource) getColumnNDV(colID int64) (ndv float64) { return ndv } -func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs) *property.StatsInfo { - if ds.tableStats == nil { - tableStats := &property.StatsInfo{ - RowCount: float64(ds.statisticTable.Count), - Cardinality: make([]float64, len(ds.Columns)), - HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), - StatsVersion: ds.statisticTable.Version, - } - if ds.statisticTable.Pseudo { - tableStats.StatsVersion = statistics.PseudoVersion - } - for i, col := range ds.Columns { - tableStats.Cardinality[i] = ds.getColumnNDV(col.ID) - } - ds.tableStats = tableStats - ds.TblColHists = ds.statisticTable.ID2UniqueID(ds.TblCols) +func (ds *DataSource) initStats() { + if ds.tableStats != nil { + return + } + tableStats := &property.StatsInfo{ + RowCount: float64(ds.statisticTable.Count), + Cardinality: make([]float64, len(ds.Columns)), + HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), + StatsVersion: ds.statisticTable.Version, } - selectivity, nodes, err := ds.tableStats.HistColl.Selectivity(ds.ctx, conds) + if ds.statisticTable.Pseudo { + tableStats.StatsVersion = statistics.PseudoVersion + } + for i, col := range ds.Columns { + tableStats.Cardinality[i] = ds.getColumnNDV(col.ID) + } + ds.tableStats = tableStats + ds.TblColHists = ds.statisticTable.ID2UniqueID(ds.TblCols) +} + +func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs, filledPaths []*util.AccessPath) *property.StatsInfo { + ds.initStats() + selectivity, nodes, err := ds.tableStats.HistColl.Selectivity(ds.ctx, conds, filledPaths) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) selectivity = selectionFactor @@ -168,31 +174,38 @@ func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs) *property.S // DeriveStats implement LogicalPlan DeriveStats interface. func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema) (*property.StatsInfo, error) { + ds.initStats() // PushDownNot here can convert query 'not (a != 1)' to 'a = 1'. for i, expr := range ds.pushedDownConds { ds.pushedDownConds[i] = expression.PushDownNot(ds.ctx, expr) } - ds.stats = ds.deriveStatsByFilter(ds.pushedDownConds) for _, path := range ds.possibleAccessPaths { - if path.isTablePath { + if path.IsTablePath { + continue + } + err := ds.fillIndexPath(path, ds.pushedDownConds) + if err != nil { + return nil, err + } + } + ds.stats = ds.deriveStatsByFilter(ds.pushedDownConds, ds.possibleAccessPaths) + for _, path := range ds.possibleAccessPaths { + if path.IsTablePath { noIntervalRanges, err := ds.deriveTablePathStats(path, ds.pushedDownConds, false) if err != nil { return nil, err } // If we have point or empty range, just remove other possible paths. - if noIntervalRanges || len(path.ranges) == 0 { + if noIntervalRanges || len(path.Ranges) == 0 { ds.possibleAccessPaths[0] = path ds.possibleAccessPaths = ds.possibleAccessPaths[:1] break } continue } - noIntervalRanges, err := ds.deriveIndexPathStats(path, ds.pushedDownConds, false) - if err != nil { - return nil, err - } + noIntervalRanges := ds.deriveIndexPathStats(path, ds.pushedDownConds, false) // If we have empty range, or point range on unique index, just remove other possible paths. - if (noIntervalRanges && path.index.Unique) || len(path.ranges) == 0 { + if (noIntervalRanges && path.Index.Unique) || len(path.Ranges) == 0 { ds.possibleAccessPaths[0] = path ds.possibleAccessPaths = ds.possibleAccessPaths[:1] break @@ -204,7 +217,7 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * // If there is an index path, we current do not consider `IndexMergePath`. needConsiderIndexMerge := true for i := 1; i < len(ds.possibleAccessPaths); i++ { - if len(ds.possibleAccessPaths[i].accessConds) != 0 { + if len(ds.possibleAccessPaths[i].AccessConds) != 0 { needConsiderIndexMerge = false break } @@ -243,7 +256,7 @@ func (ts *LogicalTableScan) DeriveStats(childStats []*property.StatsInfo, selfSc // `PushDownNot` function call in multiple `DeriveStats` then. ts.AccessConds[i] = expression.PushDownNot(ts.ctx, expr) } - ts.stats = ts.Source.deriveStatsByFilter(ts.AccessConds) + ts.stats = ts.Source.deriveStatsByFilter(ts.AccessConds, nil) sc := ts.SCtx().GetSessionVars().StmtCtx // ts.Handle could be nil if PK is Handle, and PK column has been pruned. if ts.Handle != nil { @@ -268,7 +281,7 @@ func (is *LogicalIndexScan) DeriveStats(childStats []*property.StatsInfo, selfSc for i, expr := range is.AccessConds { is.AccessConds[i] = expression.PushDownNot(is.ctx, expr) } - is.stats = is.Source.deriveStatsByFilter(is.AccessConds) + is.stats = is.Source.deriveStatsByFilter(is.AccessConds, nil) if len(is.AccessConds) == 0 { is.Ranges = ranger.FullRange() } @@ -294,7 +307,7 @@ func (ds *DataSource) generateIndexMergeOrPaths() { if !ok || sf.FuncName.L != ast.LogicOr { continue } - var partialPaths = make([]*accessPath, 0, usedIndexCount) + var partialPaths = make([]*util.AccessPath, 0, usedIndexCount) dnfItems := expression.FlattenDNFConditions(sf) for _, item := range dnfItems { cnfItems := expression.SplitCNFItems(item) @@ -336,49 +349,50 @@ func (ds *DataSource) isInIndexMergeHints(name string) bool { } // accessPathsForConds generates all possible index paths for conditions. -func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, usedIndexCount int) []*accessPath { - var results = make([]*accessPath, 0, usedIndexCount) +func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, usedIndexCount int) []*util.AccessPath { + var results = make([]*util.AccessPath, 0, usedIndexCount) for i := 0; i < usedIndexCount; i++ { - path := &accessPath{} - if ds.possibleAccessPaths[i].isTablePath { + path := &util.AccessPath{} + if ds.possibleAccessPaths[i].IsTablePath { if !ds.isInIndexMergeHints("primary") { continue } - path.isTablePath = true + path.IsTablePath = true noIntervalRanges, err := ds.deriveTablePathStats(path, conditions, true) if err != nil { logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) continue } // If we have point or empty range, just remove other possible paths. - if noIntervalRanges || len(path.ranges) == 0 { + if noIntervalRanges || len(path.Ranges) == 0 { results[0] = path results = results[:1] break } } else { - path.index = ds.possibleAccessPaths[i].index - if !ds.isInIndexMergeHints(path.index.Name.L) { + path.Index = ds.possibleAccessPaths[i].Index + if !ds.isInIndexMergeHints(path.Index.Name.L) { continue } - noIntervalRanges, err := ds.deriveIndexPathStats(path, conditions, true) + err := ds.fillIndexPath(path, conditions) if err != nil { logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) continue } + noIntervalRanges := ds.deriveIndexPathStats(path, conditions, true) // If we have empty range, or point range on unique index, just remove other possible paths. - if (noIntervalRanges && path.index.Unique) || len(path.ranges) == 0 { + if (noIntervalRanges && path.Index.Unique) || len(path.Ranges) == 0 { results[0] = path results = results[:1] break } } - // If accessConds is empty or tableFilter is not empty, we ignore the access path. + // If AccessConds is empty or tableFilter is not empty, we ignore the access path. // Now these conditions are too strict. // For example, a sql `select * from t where a > 1 or (b < 2 and c > 3)` and table `t` with indexes // on a and b separately. we can generate a `IndexMergePath` with table filter `a > 1 or (b < 2 and c > 3)`. // TODO: solve the above case - if len(path.tableFilters) > 0 || len(path.accessConds) == 0 { + if len(path.TableFilters) > 0 || len(path.AccessConds) == 0 { continue } results = append(results, path) @@ -392,15 +406,15 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us // with most columns, e.g, filter is c > 1 and the input indexes are c and c_d_e, // the former one is enough, and it is less expensive in execution compared with the latter one. // TODO: improve strategy of the partial path selection -func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*accessPath) *accessPath { +func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*util.AccessPath) *util.AccessPath { if len(indexAccessPaths) == 1 { return indexAccessPaths[0] } maxColsIndex := 0 - maxCols := len(indexAccessPaths[0].idxCols) + maxCols := len(indexAccessPaths[0].IdxCols) for i := 1; i < len(indexAccessPaths); i++ { - current := len(indexAccessPaths[i].idxCols) + current := len(indexAccessPaths[i].IdxCols) if current > maxCols { maxColsIndex = i maxCols = current @@ -410,10 +424,10 @@ func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*accessPath) } // buildIndexMergeOrPath generates one possible IndexMergePath. -func (ds *DataSource) buildIndexMergeOrPath(partialPaths []*accessPath, current int) *accessPath { - indexMergePath := &accessPath{partialIndexPaths: partialPaths} - indexMergePath.tableFilters = append(indexMergePath.tableFilters, ds.pushedDownConds[:current]...) - indexMergePath.tableFilters = append(indexMergePath.tableFilters, ds.pushedDownConds[current+1:]...) +func (ds *DataSource) buildIndexMergeOrPath(partialPaths []*util.AccessPath, current int) *util.AccessPath { + indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths} + indexMergePath.TableFilters = append(indexMergePath.TableFilters, ds.pushedDownConds[:current]...) + indexMergePath.TableFilters = append(indexMergePath.TableFilters, ds.pushedDownConds[current+1:]...) return indexMergePath } diff --git a/planner/util/path.go b/planner/util/path.go new file mode 100644 index 0000000000000..5f12f9231eeb0 --- /dev/null +++ b/planner/util/path.go @@ -0,0 +1,118 @@ +// Copyright 2019 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/ranger" +) + +// AccessPath indicates the way we access a table: by using single index, or by using multiple indexes, +// or just by using table scan. +type AccessPath struct { + Index *model.IndexInfo + FullIdxCols []*expression.Column + FullIdxColLens []int + IdxCols []*expression.Column + IdxColLens []int + Ranges []*ranger.Range + // CountAfterAccess is the row count after we apply range seek and before we use other filter to filter data. + CountAfterAccess float64 + // CountAfterIndex is the row count after we apply filters on index and before we apply the table filters. + CountAfterIndex float64 + AccessConds []expression.Expression + EqCondCount int + EqOrInCondCount int + IndexFilters []expression.Expression + TableFilters []expression.Expression + // IsTablePath indicates whether this path is table path. + IsTablePath bool + // Forced means this path is generated by `use/force index()`. + Forced bool + // PartialIndexPaths store all index access paths. + // If there are extra filters, store them in TableFilters. + PartialIndexPaths []*AccessPath + + IsDNFCond bool + + StoreType kv.StoreType +} + +// SplitCorColAccessCondFromFilters move the necessary filter in the form of index_col = corrlated_col to access conditions. +func (path *AccessPath) SplitCorColAccessCondFromFilters(eqOrInCount int) (access, remained []expression.Expression) { + access = make([]expression.Expression, len(path.IdxCols)-eqOrInCount) + used := make([]bool, len(path.TableFilters)) + for i := eqOrInCount; i < len(path.IdxCols); i++ { + matched := false + for j, filter := range path.TableFilters { + if used[j] || !isColEqCorColOrConstant(filter, path.IdxCols[i]) { + continue + } + matched = true + access[i-eqOrInCount] = filter + if path.IdxColLens[i] == types.UnspecifiedLength { + used[j] = true + } + break + } + if !matched { + access = access[:i-eqOrInCount] + break + } + } + for i, ok := range used { + if !ok { + remained = append(remained, path.TableFilters[i]) + } + } + return access, remained +} + +// isColEqCorColOrConstant checks if the expression is a eq function that one side is constant or correlated column +// and another is column. +func isColEqCorColOrConstant(filter expression.Expression, col *expression.Column) bool { + f, ok := filter.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { + return false + } + if c, ok := f.GetArgs()[0].(*expression.Column); ok { + if _, ok := f.GetArgs()[1].(*expression.Constant); ok { + if col.Equal(nil, c) { + return true + } + } + if _, ok := f.GetArgs()[1].(*expression.CorrelatedColumn); ok { + if col.Equal(nil, c) { + return true + } + } + } + if c, ok := f.GetArgs()[1].(*expression.Column); ok { + if _, ok := f.GetArgs()[0].(*expression.Constant); ok { + if col.Equal(nil, c) { + return true + } + } + if _, ok := f.GetArgs()[0].(*expression.CorrelatedColumn); ok { + if col.Equal(nil, c) { + return true + } + } + } + return false +} diff --git a/session/session.go b/session/session.go index 424481349e89a..79307a9bb74bb 100644 --- a/session/session.go +++ b/session/session.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -367,7 +368,7 @@ func (s *session) StoreQueryFeedback(feedback interface{}) { // FieldList returns fields list of a table. func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { - is := executor.GetInfoSchema(s) + is := infoschema.GetInfoSchema(s) dbName := model.NewCIStr(s.GetSessionVars().CurrentDB) tName := model.NewCIStr(tableName) table, err := is.TableByName(dbName, tName) @@ -796,6 +797,10 @@ func (s *session) ExecRestrictedSQLWithSnapshot(sql string) ([]chunk.Row, []*ast } // Set snapshot. if snapshot != 0 { + se.sessionVars.SnapshotInfoschema, err = domain.GetDomain(s).GetSnapshotInfoSchema(snapshot) + if err != nil { + return nil, nil, err + } if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(snapshot, 10)); err != nil { return nil, nil, err } @@ -803,6 +808,7 @@ func (s *session) ExecRestrictedSQLWithSnapshot(sql string) ([]chunk.Row, []*ast if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) } + se.sessionVars.SnapshotInfoschema = nil }() } return execRestrictedSQL(ctx, se, sql) @@ -1167,7 +1173,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) s.PrepareTxnFuture(ctx) - prepareExec := executor.NewPrepareExec(s, executor.GetInfoSchema(s), sql) + prepareExec := executor.NewPrepareExec(s, infoschema.GetInfoSchema(s), sql) err = prepareExec.Next(ctx, nil) if err != nil { return @@ -1194,7 +1200,7 @@ func (s *session) CachedPlanExec(ctx context.Context, stmtID uint32, prepareStmt *plannercore.CachedPrepareStmt, args []types.Datum) (sqlexec.RecordSet, error) { prepared := prepareStmt.PreparedAst // compile ExecStmt - is := executor.GetInfoSchema(s) + is := infoschema.GetInfoSchema(s) execAst := &ast.ExecuteStmt{ExecID: stmtID} if err := executor.ResetContextOfStmt(s, execAst); err != nil { return nil, err @@ -1247,7 +1253,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. return false, nil } // check schema version - is := executor.GetInfoSchema(s) + is := infoschema.GetInfoSchema(s) if prepared.SchemaVersion != is.SchemaMetaVersion() { prepared.CachedPlan = nil return false, nil diff --git a/session/session_test.go b/session/session_test.go index bcac4686392ff..bf8502149b9fe 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -748,7 +748,7 @@ func (s *testSessionSuite) TestSessionAuth(c *C) { c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "Any not exist username with zero password!", Hostname: "anyhost"}, []byte(""), []byte("")), IsFalse) } -func (s *testSessionSuite) TestSkipWithGrant(c *C) { +func (s *testSessionSerialSuite) TestSkipWithGrant(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) save2 := privileges.SkipWithGrant diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 2c21f4fd0030f..9bff0a4df9860 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -313,30 +313,12 @@ func (sc *StatementContext) WarningCount() uint16 { return wc } -const zero = "0" - // NumErrorWarnings gets warning and error count. -func (sc *StatementContext) NumErrorWarnings() (ec, wc string) { - var ( - ecNum uint16 - wcNum int - ) +func (sc *StatementContext) NumErrorWarnings() (ec uint16, wc int) { sc.mu.Lock() - ecNum = sc.mu.errorCount - wcNum = len(sc.mu.warnings) + ec = sc.mu.errorCount + wc = len(sc.mu.warnings) sc.mu.Unlock() - - if ecNum == 0 { - ec = zero - } else { - ec = strconv.Itoa(int(ecNum)) - } - - if wcNum == 0 { - wc = zero - } else { - wc = strconv.Itoa(wcNum) - } return } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index df7ecb66679bf..abb07788a8360 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -203,6 +203,10 @@ type SessionVars struct { Users map[string]string // systems variables, don't modify it directly, use GetSystemVar/SetSystemVar method. systems map[string]string + // SysWarningCount is the system variable "warning_count", because it is on the hot path, so we extract it from the systems + SysWarningCount int + // SysErrorCount is the system variable "error_count", because it is on the hot path, so we extract it from the systems + SysErrorCount uint16 // PreparedStmts stores prepared statement. PreparedStmts map[uint32]interface{} PreparedStmtNameToID map[string]uint32 @@ -711,6 +715,11 @@ func (s *SessionVars) Location() *time.Location { // GetSystemVar gets the string value of a system variable. func (s *SessionVars) GetSystemVar(name string) (string, bool) { + if name == WarningCount { + return strconv.Itoa(s.SysWarningCount), true + } else if name == ErrorCount { + return strconv.Itoa(int(s.SysErrorCount)), true + } val, ok := s.systems[name] return val, ok } diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 6c3434be0a2fc..460a7ff48c456 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -142,7 +142,7 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { case TiDBCheckMb4ValueInUTF8: return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil } - sVal, ok := s.systems[key] + sVal, ok := s.GetSystemVar(key) if ok { return sVal, true, nil } diff --git a/statistics/feedback.go b/statistics/feedback.go index 3db3ae09e43e4..cfc371947e23c 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -302,7 +302,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } total := 0 sc := &stmtctx.StatementContext{TimeZone: time.UTC} - min, max := GetMinValue(h.Tp), GetMaxValue(h.Tp) + min, max := types.GetMinValue(h.Tp), types.GetMaxValue(h.Tp) for _, fb := range feedback.Feedback { skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) if err != nil { @@ -927,73 +927,3 @@ func SupportColumnType(ft *types.FieldType) bool { } return false } - -// GetMaxValue returns the max value datum for each type. -func GetMaxValue(ft *types.FieldType) (max types.Datum) { - switch ft.Tp { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - if mysql.HasUnsignedFlag(ft.Flag) { - max.SetUint64(types.IntergerUnsignedUpperBound(ft.Tp)) - } else { - max.SetInt64(types.IntergerSignedUpperBound(ft.Tp)) - } - case mysql.TypeFloat: - max.SetFloat32(float32(types.GetMaxFloat(ft.Flen, ft.Decimal))) - case mysql.TypeDouble: - max.SetFloat64(types.GetMaxFloat(ft.Flen, ft.Decimal)) - case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - val := types.MaxValueDatum() - bytes, err := codec.EncodeKey(nil, nil, val) - // should not happen - if err != nil { - logutil.BgLogger().Error("encode key fail", zap.Error(err)) - } - max.SetBytes(bytes) - case mysql.TypeNewDecimal: - max.SetMysqlDecimal(types.NewMaxOrMinDec(false, ft.Flen, ft.Decimal)) - case mysql.TypeDuration: - max.SetMysqlDuration(types.Duration{Duration: types.MaxTime}) - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - if ft.Tp == mysql.TypeDate || ft.Tp == mysql.TypeDatetime { - max.SetMysqlTime(types.Time{Time: types.MaxDatetime, Type: ft.Tp}) - } else { - max.SetMysqlTime(types.MaxTimestamp) - } - } - return -} - -// GetMinValue returns the min value datum for each type. -func GetMinValue(ft *types.FieldType) (min types.Datum) { - switch ft.Tp { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - if mysql.HasUnsignedFlag(ft.Flag) { - min.SetUint64(0) - } else { - min.SetInt64(types.IntergerSignedLowerBound(ft.Tp)) - } - case mysql.TypeFloat: - min.SetFloat32(float32(-types.GetMaxFloat(ft.Flen, ft.Decimal))) - case mysql.TypeDouble: - min.SetFloat64(-types.GetMaxFloat(ft.Flen, ft.Decimal)) - case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - val := types.MinNotNullDatum() - bytes, err := codec.EncodeKey(nil, nil, val) - // should not happen - if err != nil { - logutil.BgLogger().Error("encode key fail", zap.Error(err)) - } - min.SetBytes(bytes) - case mysql.TypeNewDecimal: - min.SetMysqlDecimal(types.NewMaxOrMinDec(true, ft.Flen, ft.Decimal)) - case mysql.TypeDuration: - min.SetMysqlDuration(types.Duration{Duration: types.MinTime}) - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - if ft.Tp == mysql.TypeDate || ft.Tp == mysql.TypeDatetime { - min.SetMysqlTime(types.Time{Time: types.MinDatetime, Type: ft.Tp}) - } else { - min.SetMysqlTime(types.MinTimestamp) - } - } - return -} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 766fd739f5f76..73187a49dfe44 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -948,10 +948,10 @@ func (h *Handle) dumpRangeFeedback(sc *stmtctx.StatementContext, ran *ranger.Ran return nil } if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = statistics.GetMinValue(q.Hist.Tp) + ran.LowVal[0] = types.GetMinValue(q.Hist.Tp) } if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = statistics.GetMaxValue(q.Hist.Tp) + ran.HighVal[0] = types.GetMaxValue(q.Hist.Tp) } } ranges, ok := q.Hist.SplitRange(sc, []*ranger.Range{ran}, q.Tp == statistics.IndexType) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index fef5c54907d25..d8168003812be 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + planutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" @@ -147,7 +148,7 @@ func isColEqCorCol(filter expression.Expression) *expression.Column { // The definition of selectivity is (row count after filter / row count before filter). // And exprs must be CNF now, in other words, `exprs[0] and exprs[1] and ... and exprs[len - 1]` should be held when you call this. // Currently the time complexity is o(n^2). -func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Expression) (float64, []*StatsNode, error) { +func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Expression, filledPaths []*planutil.AccessPath) (float64, []*StatsNode, error) { // If table's count is zero or conditions are empty, we should return 100% selectivity. if coll.Count == 0 || len(exprs) == 0 { return 1, nil, nil @@ -189,7 +190,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp for id, colInfo := range coll.Columns { col := expression.ColInfo2Col(extractedCols, colInfo.Info) if col != nil { - maskCovered, ranges, _, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, col) + maskCovered, ranges, _, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, nil, col) if err != nil { return 0, nil, errors.Trace(err) } @@ -211,6 +212,13 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp nodes[len(nodes)-1].Selectivity = cnt / float64(coll.Count) } } + id2Paths := make(map[int64]*planutil.AccessPath) + for _, path := range filledPaths { + if path.IsTablePath { + continue + } + id2Paths[path.Index.ID] = path + } for id, idxInfo := range coll.Indices { idxCols := expression.FindPrefixOfIndex(extractedCols, coll.Idx2ColumnIDs[id]) if len(idxCols) > 0 { @@ -218,7 +226,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp for i := 0; i < len(idxCols); i++ { lengths = append(lengths, idxInfo.Info.Columns[i].Length) } - maskCovered, ranges, partCover, err := getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, idxCols...) + maskCovered, ranges, partCover, err := getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, id2Paths[idxInfo.ID], idxCols...) if err != nil { return 0, nil, errors.Trace(err) } @@ -259,8 +267,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp return ret, nodes, nil } -func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, - lengths []int, cols ...*expression.Column) (mask int64, ranges []*ranger.Range, partCover bool, err error) { +func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) (mask int64, ranges []*ranger.Range, partCover bool, err error) { sc := ctx.GetSessionVars().StmtCtx isDNF := false var accessConds, remainedConds []expression.Expression @@ -269,9 +276,16 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].UniqueID) ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength) case ranger.IndexRangeType: + if cachedPath != nil { + ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges, cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond + break + } var res *ranger.DetachRangeResult res, err = ranger.DetachCondAndBuildRangeForIndex(ctx, exprs, cols, lengths) ranges, accessConds, remainedConds, isDNF = res.Ranges, res.AccessConds, res.RemainedConds, res.IsDNFCond + if err != nil { + return 0, nil, false, err + } default: panic("should never be here") } diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 109b4df605946..20a56f600ee42 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -301,12 +301,12 @@ func (s *testStatsSuite) TestSelectivity(c *C) { histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) - ratio, _, err := histColl.Selectivity(sctx, sel.Conditions) + ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) histColl.Count *= 10 - ratio, _, err = histColl.Selectivity(sctx, sel.Conditions) + ratio, _, err = histColl.Selectivity(sctx, sel.Conditions, nil) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) } @@ -531,7 +531,7 @@ func BenchmarkSelectivity(b *testing.B) { b.Run("Selectivity", func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := statsTbl.Selectivity(sctx, p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection).Conditions) + _, _, err := statsTbl.Selectivity(sctx, p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection).Conditions, nil) c.Assert(err, IsNil) } b.ReportAllocs() diff --git a/store/tikv/client.go b/store/tikv/client.go index 24635272243fa..3316de6b39de5 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -80,21 +80,23 @@ type connArray struct { streamTimeout chan *tikvrpc.Lease // batchConn is not null when batch is enabled. *batchConn + done chan struct{} } -func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32, done <-chan struct{}) (*connArray, error) { +func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32) (*connArray, error) { a := &connArray{ index: 0, v: make([]*grpc.ClientConn, maxSize), streamTimeout: make(chan *tikvrpc.Lease, 1024), + done: make(chan struct{}), } - if err := a.Init(addr, security, idleNotify, done); err != nil { + if err := a.Init(addr, security, idleNotify); err != nil { return nil, err } return a, nil } -func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, done <-chan struct{}) error { +func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32) error { a.target = addr opt := grpc.WithInsecure() @@ -162,7 +164,7 @@ func (a *connArray) Init(addr string, security config.Security, idleNotify *uint a.batchCommandsClients = append(a.batchCommandsClients, batchClient) } } - go tikvrpc.CheckStreamTimeoutLoop(a.streamTimeout, done) + go tikvrpc.CheckStreamTimeoutLoop(a.streamTimeout, a.done) if allowBatch { go a.batchSendLoop(cfg.TiKVClient) } @@ -187,6 +189,8 @@ func (a *connArray) Close() { a.v[i] = nil } } + + close(a.done) } // rpcClient is RPC client struct. @@ -195,7 +199,6 @@ func (a *connArray) Close() { // that there are too many concurrent requests which overload the service of TiKV. type rpcClient struct { sync.RWMutex - done chan struct{} conns map[string]*connArray security config.Security @@ -208,7 +211,6 @@ type rpcClient struct { func newRPCClient(security config.Security) *rpcClient { return &rpcClient{ - done: make(chan struct{}, 1), conns: make(map[string]*connArray), security: security, } @@ -244,7 +246,7 @@ func (c *rpcClient) createConnArray(addr string) (*connArray, error) { if !ok { var err error connCount := config.GetGlobalConfig().TiKVClient.GrpcConnectionCount - array, err = newConnArray(connCount, addr, c.security, &c.idleNotify, c.done) + array, err = newConnArray(connCount, addr, c.security, &c.idleNotify) if err != nil { return nil, err } @@ -352,7 +354,6 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R func (c *rpcClient) Close() error { // TODO: add a unit test for SendRequest After Closed - close(c.done) c.closeConns() return nil } diff --git a/store/tikv/oracle/oracles/pd.go b/store/tikv/oracle/oracles/pd.go index 3d07426ed8833..e0c921973be08 100644 --- a/store/tikv/oracle/oracles/pd.go +++ b/store/tikv/oracle/oracles/pd.go @@ -121,6 +121,7 @@ func (o *pdOracle) setLastTS(ts uint64) { func (o *pdOracle) updateTS(ctx context.Context, interval time.Duration) { ticker := time.NewTicker(interval) + defer ticker.Stop() for { select { case <-ticker.C: @@ -131,7 +132,6 @@ func (o *pdOracle) updateTS(ctx context.Context, interval time.Duration) { } o.setLastTS(ts) case <-o.quit: - ticker.Stop() return } } diff --git a/table/column.go b/table/column.go index 32f439266d7f1..aee631fee7b91 100644 --- a/table/column.go +++ b/table/column.go @@ -232,18 +232,6 @@ type ColDesc struct { const defaultPrivileges = "select,insert,update,references" -// GetTypeDesc gets the description for column type. -func (c *Column) GetTypeDesc() string { - desc := c.FieldType.CompactStr() - if mysql.HasUnsignedFlag(c.Flag) && c.Tp != mysql.TypeBit && c.Tp != mysql.TypeYear { - desc += " unsigned" - } - if mysql.HasZerofillFlag(c.Flag) && c.Tp != mysql.TypeYear { - desc += " zerofill" - } - return desc -} - // NewColDesc returns a new ColDesc for a column. func NewColDesc(col *Column) *ColDesc { // TODO: if we have no primary key and a unique index which's columns are all not null diff --git a/types/datum.go b/types/datum.go index 0b3e9148d1809..525ce3be7b2b4 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1878,3 +1878,182 @@ func CloneRow(dr []Datum) []Datum { } return c } + +// GetMaxValue returns the max value datum for each type. +func GetMaxValue(ft *FieldType) (max Datum) { + switch ft.Tp { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + if mysql.HasUnsignedFlag(ft.Flag) { + max.SetUint64(IntergerUnsignedUpperBound(ft.Tp)) + } else { + max.SetInt64(IntergerSignedUpperBound(ft.Tp)) + } + case mysql.TypeFloat: + max.SetFloat32(float32(GetMaxFloat(ft.Flen, ft.Decimal))) + case mysql.TypeDouble: + max.SetFloat64(GetMaxFloat(ft.Flen, ft.Decimal)) + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + // codec.Encode KindMaxValue, to avoid import circle + bytes := []byte{250} + max.SetBytes(bytes) + case mysql.TypeNewDecimal: + max.SetMysqlDecimal(NewMaxOrMinDec(false, ft.Flen, ft.Decimal)) + case mysql.TypeDuration: + max.SetMysqlDuration(Duration{Duration: MaxTime}) + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: + if ft.Tp == mysql.TypeDate || ft.Tp == mysql.TypeDatetime { + max.SetMysqlTime(Time{Time: MaxDatetime, Type: ft.Tp}) + } else { + max.SetMysqlTime(MaxTimestamp) + } + } + return +} + +// GetMinValue returns the min value datum for each type. +func GetMinValue(ft *FieldType) (min Datum) { + switch ft.Tp { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + if mysql.HasUnsignedFlag(ft.Flag) { + min.SetUint64(0) + } else { + min.SetInt64(IntergerSignedLowerBound(ft.Tp)) + } + case mysql.TypeFloat: + min.SetFloat32(float32(-GetMaxFloat(ft.Flen, ft.Decimal))) + case mysql.TypeDouble: + min.SetFloat64(-GetMaxFloat(ft.Flen, ft.Decimal)) + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: + // codec.Encode KindMinNotNull, to avoid import circle + bytes := []byte{1} + min.SetBytes(bytes) + case mysql.TypeNewDecimal: + min.SetMysqlDecimal(NewMaxOrMinDec(true, ft.Flen, ft.Decimal)) + case mysql.TypeDuration: + min.SetMysqlDuration(Duration{Duration: MinTime}) + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: + if ft.Tp == mysql.TypeDate || ft.Tp == mysql.TypeDatetime { + min.SetMysqlTime(Time{Time: MinDatetime, Type: ft.Tp}) + } else { + min.SetMysqlTime(MinTimestamp) + } + } + return +} + +// RoundingType is used to indicate the rounding type for reversing evaluation. +type RoundingType uint8 + +const ( + // Ceiling means rounding up. + Ceiling RoundingType = iota + // Floor means rounding down. + Floor +) + +func getDatumBound(retType *FieldType, rType RoundingType) Datum { + if rType == Ceiling { + return GetMaxValue(retType) + } + return GetMinValue(retType) +} + +// ChangeReverseResultByUpperLowerBound is for expression's reverse evaluation. +// Here is an example for what's effort for the function: CastRealAsInt(t.a), +// if the type of column `t.a` is mysql.TypeDouble, and there is a row that t.a == MaxFloat64 +// then the cast function will arrive a result MaxInt64. But when we do the reverse evaluation, +// if the result is MaxInt64, and the rounding type is ceiling. Then we should get the MaxFloat64 +// instead of float64(MaxInt64). +// Another example: cast(1.1 as signed) = 1, +// when we get the answer 1, we can only reversely evaluate 1.0 as the column value. So in this +// case, we should judge whether the rounding type are ceiling. If it is, then we should plus one for +// 1.0 and get the reverse result 2.0. +func ChangeReverseResultByUpperLowerBound( + sc *stmtctx.StatementContext, + retType *FieldType, + res Datum, + rType RoundingType) (Datum, error) { + d, err := res.ConvertTo(sc, retType) + if terror.ErrorEqual(err, ErrOverflow) { + return d, nil + } + if err != nil { + return d, err + } + resRetType := FieldType{} + switch res.Kind() { + case KindInt64: + resRetType.Tp = mysql.TypeLonglong + case KindUint64: + resRetType.Tp = mysql.TypeLonglong + resRetType.Flag |= mysql.UnsignedFlag + case KindFloat32: + resRetType.Tp = mysql.TypeFloat + case KindFloat64: + resRetType.Tp = mysql.TypeDouble + case KindMysqlDecimal: + resRetType.Tp = mysql.TypeNewDecimal + resRetType.Flen = int(res.GetMysqlDecimal().GetDigitsFrac() + res.GetMysqlDecimal().GetDigitsInt()) + resRetType.Decimal = int(res.GetMysqlDecimal().GetDigitsInt()) + } + bound := getDatumBound(&resRetType, rType) + cmp, err := d.CompareDatum(sc, &bound) + if err != nil { + return d, err + } + if cmp == 0 { + d = getDatumBound(retType, rType) + } else if rType == Ceiling { + switch retType.Tp { + case mysql.TypeShort: + if mysql.HasUnsignedFlag(retType.Flag) { + if d.GetUint64() != math.MaxUint16 { + d.SetUint64(d.GetUint64() + 1) + } + } else { + if d.GetInt64() != math.MaxInt16 { + d.SetInt64(d.GetInt64() + 1) + } + } + case mysql.TypeLong: + if mysql.HasUnsignedFlag(retType.Flag) { + if d.GetUint64() != math.MaxUint32 { + d.SetUint64(d.GetUint64() + 1) + } + } else { + if d.GetInt64() != math.MaxInt32 { + d.SetInt64(d.GetInt64() + 1) + } + } + case mysql.TypeLonglong: + if mysql.HasUnsignedFlag(retType.Flag) { + if d.GetUint64() != math.MaxUint64 { + d.SetUint64(d.GetUint64() + 1) + } + } else { + if d.GetInt64() != math.MaxInt64 { + d.SetInt64(d.GetInt64() + 1) + } + } + case mysql.TypeFloat: + if d.GetFloat32() != math.MaxFloat32 { + d.SetFloat32(d.GetFloat32() + 1.0) + } + case mysql.TypeDouble: + if d.GetFloat64() != math.MaxFloat64 { + d.SetFloat64(d.GetFloat64() + 1.0) + } + case mysql.TypeNewDecimal: + if d.GetMysqlDecimal().Compare(NewMaxOrMinDec(false, retType.Flen, retType.Decimal)) != 0 { + var decimalOne, newD MyDecimal + one := decimalOne.FromInt(1) + err = DecimalAdd(d.GetMysqlDecimal(), one, &newD) + if err != nil { + return d, err + } + d = NewDecimalDatum(&newD) + } + } + } + return d, nil +} diff --git a/types/datum_test.go b/types/datum_test.go index 40d6ae0dd36a0..09b612e47e681 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -15,7 +15,9 @@ package types import ( "fmt" + "math" "reflect" + "strconv" "testing" "time" @@ -373,6 +375,124 @@ func (ts *testDatumSuite) TestCloneDatum(c *C) { } } +func newTypeWithFlag(tp byte, flag uint) *FieldType { + t := NewFieldType(tp) + t.Flag |= flag + return t +} + +func newMyDecimal(val string, c *C) *MyDecimal { + t := MyDecimal{} + err := t.FromString([]byte(val)) + c.Assert(err, IsNil) + return &t +} + +func newRetTypeWithFlenDecimal(tp byte, flen int, decimal int) *FieldType { + return &FieldType{ + Tp: tp, + Flen: flen, + Decimal: decimal, + } +} + +func (ts *testDatumSuite) TestChangeReverseResultByUpperLowerBound(c *C) { + sc := new(stmtctx.StatementContext) + sc.IgnoreTruncate = true + sc.OverflowAsWarning = true + // TODO: add more reserve convert tests for each pair of convert type. + testData := []struct { + a Datum + res Datum + retType *FieldType + roundType RoundingType + }{ + // int64 reserve to uint64 + { + NewIntDatum(1), + NewUintDatum(2), + newTypeWithFlag(mysql.TypeLonglong, mysql.UnsignedFlag), + Ceiling, + }, + { + NewIntDatum(1), + NewUintDatum(1), + newTypeWithFlag(mysql.TypeLonglong, mysql.UnsignedFlag), + Floor, + }, + { + NewIntDatum(math.MaxInt64), + NewUintDatum(math.MaxUint64), + newTypeWithFlag(mysql.TypeLonglong, mysql.UnsignedFlag), + Ceiling, + }, + { + NewIntDatum(math.MaxInt64), + NewUintDatum(math.MaxInt64), + newTypeWithFlag(mysql.TypeLonglong, mysql.UnsignedFlag), + Floor, + }, + // int64 reserve to float64 + { + NewIntDatum(1), + NewFloat64Datum(2), + newRetTypeWithFlenDecimal(mysql.TypeDouble, mysql.MaxRealWidth, UnspecifiedLength), + Ceiling, + }, + { + NewIntDatum(1), + NewFloat64Datum(1), + newRetTypeWithFlenDecimal(mysql.TypeDouble, mysql.MaxRealWidth, UnspecifiedLength), + Floor, + }, + { + NewIntDatum(math.MaxInt64), + GetMaxValue(newRetTypeWithFlenDecimal(mysql.TypeDouble, mysql.MaxRealWidth, UnspecifiedLength)), + newRetTypeWithFlenDecimal(mysql.TypeDouble, mysql.MaxRealWidth, UnspecifiedLength), + Ceiling, + }, + { + NewIntDatum(math.MaxInt64), + NewFloat64Datum(float64(math.MaxInt64)), + newRetTypeWithFlenDecimal(mysql.TypeDouble, mysql.MaxRealWidth, UnspecifiedLength), + Floor, + }, + // int64 reserve to Decimal + { + NewIntDatum(1), + NewDecimalDatum(newMyDecimal("2", c)), + newRetTypeWithFlenDecimal(mysql.TypeNewDecimal, 30, 3), + Ceiling, + }, + { + NewIntDatum(1), + NewDecimalDatum(newMyDecimal("1", c)), + newRetTypeWithFlenDecimal(mysql.TypeNewDecimal, 30, 3), + Floor, + }, + { + NewIntDatum(math.MaxInt64), + GetMaxValue(newRetTypeWithFlenDecimal(mysql.TypeNewDecimal, 30, 3)), + newRetTypeWithFlenDecimal(mysql.TypeNewDecimal, 30, 3), + Ceiling, + }, + { + NewIntDatum(math.MaxInt64), + NewDecimalDatum(newMyDecimal(strconv.FormatInt(math.MaxInt64, 10), c)), + newRetTypeWithFlenDecimal(mysql.TypeNewDecimal, 30, 3), + Floor, + }, + } + for ith, test := range testData { + reverseRes, err := ChangeReverseResultByUpperLowerBound(sc, test.retType, test.a, test.roundType) + c.Assert(err, IsNil) + var cmp int + cmp, err = reverseRes.CompareDatum(sc, &test.res) + c.Assert(err, IsNil) + c.Assert(cmp, Equals, 0, Commentf("%dth got:%#v, expect:%#v", ith, reverseRes, test.res)) + } +} + func prepareCompareDatums() ([]Datum, []Datum) { vals := make([]Datum, 0, 5) vals = append(vals, NewIntDatum(1)) diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 4a0cfd50d82ba..b936b0eafd765 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -19,7 +19,6 @@ import ( "encoding/hex" "fmt" "sort" - "strconv" "unicode/utf8" "unsafe" @@ -55,33 +54,33 @@ func (bj BinaryJSON) Type() string { } } -// Quote is for JSON_QUOTE -func (bj BinaryJSON) Quote() string { - str := hack.String(bj.GetString()) - return strconv.Quote(string(str)) -} - // Unquote is for JSON_UNQUOTE. func (bj BinaryJSON) Unquote() (string, error) { switch bj.TypeCode { case TypeCodeString: - tmp := string(hack.String(bj.GetString())) - tlen := len(tmp) - if tlen < 2 { - return tmp, nil - } - head, tail := tmp[0], tmp[tlen-1] - if head == '"' && tail == '"' { - // Remove prefix and suffix '"' before unquoting - return unquoteString(tmp[1 : tlen-1]) - } - // if value is not double quoted, do nothing - return tmp, nil + str := string(hack.String(bj.GetString())) + return UnquoteString(str) default: return bj.String(), nil } } +// UnquoteString remove quotes in a string, +// including the quotes at the head and tail of string. +func UnquoteString(str string) (string, error) { + strLen := len(str) + if strLen < 2 { + return str, nil + } + head, tail := str[0], str[strLen-1] + if head == '"' && tail == '"' { + // Remove prefix and suffix '"' before unquoting + return unquoteString(str[1 : strLen-1]) + } + // if value is not double quoted, do nothing + return str, nil +} + // unquoteString recognizes the escape sequences shown in: // https://dev.mysql.com/doc/refman/5.7/en/json-modification-functions.html#json-unquote-character-escape-sequences func unquoteString(s string) (string, error) { diff --git a/types/mydecimal.go b/types/mydecimal.go index 5085cbedd1505..4e4ffcd5e84aa 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -250,6 +250,11 @@ func (d *MyDecimal) GetDigitsFrac() int8 { return d.digitsFrac } +// GetDigitsInt returns the digitsInt. +func (d *MyDecimal) GetDigitsInt() int8 { + return d.digitsInt +} + // String returns the decimal string representation rounded to resultFrac. func (d *MyDecimal) String() string { tmp := *d diff --git a/util/expensivequery/expensivequery.go b/util/expensivequery/expensivequery.go index 9e1b26a1cc234..6ba77fb2052f6 100644 --- a/util/expensivequery/expensivequery.go +++ b/util/expensivequery/expensivequery.go @@ -52,6 +52,7 @@ func (eqh *Handle) Run() { // use 100ms as tickInterval temply, may use given interval or use defined variable later tickInterval := time.Millisecond * time.Duration(100) ticker := time.NewTicker(tickInterval) + defer ticker.Stop() for { select { case <-ticker.C: