Skip to content

Commit

Permalink
Merge branch 'master' into composite-mvindex
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Jan 10, 2023
2 parents 181b6bd + e2a14ce commit fb81453
Show file tree
Hide file tree
Showing 10 changed files with 230 additions and 44 deletions.
26 changes: 23 additions & 3 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,11 +280,11 @@ func TestPreparedLimitOffset(t *testing.T) {
r.Check(testkit.Rows("2"))

tk.MustExec(`set @a=1.1`)
r = tk.MustQuery(`execute stmt_test_1 using @a, @b;`)
r.Check(testkit.Rows("2"))
_, err := tk.Exec(`execute stmt_test_1 using @a, @b;`)
require.True(t, plannercore.ErrWrongArguments.Equal(err))

tk.MustExec(`set @c="-1"`)
_, err := tk.Exec("execute stmt_test_1 using @c, @c")
_, err = tk.Exec("execute stmt_test_1 using @c, @c")
require.True(t, plannercore.ErrWrongArguments.Equal(err))

stmtID, _, _, err := tk.Session().PrepareStmt("select id from prepare_test limit ?")
Expand Down Expand Up @@ -767,3 +767,23 @@ func TestPreparedIssue17419(t *testing.T) {
// _, ok := tk1.Session().ShowProcess().Plan.(*plannercore.Execute)
// require.True(t, ok)
}

func TestLimitUnsupportedCase(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, key(a))")
tk.MustExec("prepare stmt from 'select * from t limit ?'")

tk.MustExec("set @a = 1.2")
tk.MustGetErrMsg("execute stmt using @a", "[planner:1210]Incorrect arguments to LIMIT")
tk.MustExec("set @a = 1.")
tk.MustGetErrMsg("execute stmt using @a", "[planner:1210]Incorrect arguments to LIMIT")
tk.MustExec("set @a = '0'")
tk.MustGetErrMsg("execute stmt using @a", "[planner:1210]Incorrect arguments to LIMIT")
tk.MustExec("set @a = '1'")
tk.MustGetErrMsg("execute stmt using @a", "[planner:1210]Incorrect arguments to LIMIT")
tk.MustExec("set @a = 1_2")
tk.MustGetErrMsg("execute stmt using @a", "[planner:1210]Incorrect arguments to LIMIT")
}
6 changes: 6 additions & 0 deletions parser/ast/dml.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,6 +358,8 @@ const (
HintUse IndexHintType = iota + 1
HintIgnore
HintForce
HintKeepOrder
HintNoKeepOrder
)

// IndexHintScope is the type for index hint for join, order by or group by.
Expand Down Expand Up @@ -388,6 +390,10 @@ func (n *IndexHint) Restore(ctx *format.RestoreCtx) error {
indexHintType = "IGNORE INDEX"
case HintForce:
indexHintType = "FORCE INDEX"
case HintKeepOrder:
indexHintType = "KEEP ORDER"
case HintNoKeepOrder:
indexHintType = "NO KEEP ORDER"
default: // Prevent accidents
return errors.New("IndexHintType has an error while matching")
}
Expand Down
16 changes: 16 additions & 0 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1436,6 +1436,14 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty,
if !prop.IsSortItemEmpty() && !candidate.isMatchProp {
return invalidTask, nil
}
// If we need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if prop.IsSortItemEmpty() && candidate.path.ForceKeepOrder {
return invalidTask, nil
}
// If we don't need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if !prop.IsSortItemEmpty() && candidate.path.ForceNoKeepOrder {
return invalidTask, nil
}
path := candidate.path
is := ds.getOriginalPhysicalIndexScan(prop, path, candidate.isMatchProp, candidate.path.IsSingleScan)
cop := &copTask{
Expand Down Expand Up @@ -1975,6 +1983,14 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid
if !prop.IsSortItemEmpty() && !candidate.isMatchProp {
return invalidTask, nil
}
// If we need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if prop.IsSortItemEmpty() && candidate.path.ForceKeepOrder {
return invalidTask, nil
}
// If we don't need to keep order for the index scan, we should forbid the non-keep-order index scan when we try to generate the path.
if !prop.IsSortItemEmpty() && candidate.path.ForceNoKeepOrder {
return invalidTask, nil
}
ts, _ := ds.getOriginalPhysicalTableScan(prop, candidate.path, candidate.isMatchProp)
if ts.KeepOrder && ts.StoreType == kv.TiFlash && (ts.Desc || ds.SCtx().GetSessionVars().TiFlashFastScan) {
// TiFlash fast mode(https://github.com/pingcap/tidb/pull/35851) does not keep order in TableScan
Expand Down
45 changes: 45 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1324,6 +1324,43 @@ func TestReadFromStorageHint(t *testing.T) {
}
}

func TestKeepOrderHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("set tidb_cost_model_version=2")
tk.MustExec("drop table if exists t, t1, t2")
tk.MustExec("create table t(a int, b int, primary key(a));")
tk.MustExec("create table t1(a int, b int, index idx_a(a), index idx_b(b));")

// If the optimizer can not generate the keep order plan, it will report error
err := tk.ExecToErr("explain select /*+ keep_order(t1, idx_a) */ * from t1 where a<10 limit 1;")
require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query")

err = tk.ExecToErr("explain select /*+ keep_order(t, primary) */ * from t where a<10 limit 1;")
require.EqualError(t, err, "[planner:1815]Internal : Can't find a proper physical plan for this query")

var input []string
var output []struct {
SQL string
Plan []string
Warn []string
}
integrationSuiteData := core.GetIntegrationSuiteData()
integrationSuiteData.LoadTestCases(t, &input, &output)
for i, tt := range input {
testdata.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows())
output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())
})
res := tk.MustQuery(tt)
res.Check(testkit.Rows(output[i].Plan...))
require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))
}
}

func TestViewHint(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -8219,3 +8256,11 @@ func TestAutoIncrementCheckWithCheckConstraint(t *testing.T) {
KEY idx_autoinc_id (id)
)`)
}

func TestIssue40285(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("CREATE TABLE t(col1 enum('p5', '9a33x') NOT NULL DEFAULT 'p5',col2 tinyblob DEFAULT NULL) ENGINE = InnoDB DEFAULT CHARSET = latin1 COLLATE = latin1_bin;")
tk.MustQuery("(select last_value(col1) over () as r0 from t) union all (select col2 as r0 from t);")
}
89 changes: 49 additions & 40 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,10 @@ const (
HintIgnoreIndex = "ignore_index"
// HintForceIndex make optimizer to use this index even if it thinks a table scan is more efficient.
HintForceIndex = "force_index"
// HintKeepOrder is hint enforce using some indexes and keep the index's order.
HintKeepOrder = "keep_order"
// HintNoKeepOrder is hint enforce using some indexes and not keep the index's order.
HintNoKeepOrder = "no_keep_order"
// HintAggToCop is hint enforce pushing aggregation to coprocessor.
HintAggToCop = "agg_to_cop"
// HintReadFromStorage is hint enforce some tables read from specific type of storage.
Expand Down Expand Up @@ -162,7 +166,7 @@ func (a *aggOrderByResolver) Enter(inNode ast.Node) (ast.Node, bool) {
a.exprDepth++
if n, ok := inNode.(*driver.ParamMarkerExpr); ok {
if a.exprDepth == 1 {
_, isNull, isExpectedType := getUintFromNode(a.ctx, n)
_, isNull, isExpectedType := getUintFromNode(a.ctx, n, false)
// For constant uint expression in top level, it should be treated as position expression.
if !isNull && isExpectedType {
return expression.ConstructPositionExpr(n), true
Expand Down Expand Up @@ -1576,7 +1580,9 @@ func (*PlanBuilder) setUnionFlen(resultTp *types.FieldType, cols []expression.Ex
childTp := cols[i].GetType()
childTpCharLen := 1
if isBinary {
childTpCharLen = charset.CharacterSetInfos[childTp.GetCharset()].Maxlen
if charsetInfo, ok := charset.CharacterSetInfos[childTp.GetCharset()]; ok {
childTpCharLen = charsetInfo.Maxlen
}
}
resultTp.SetFlen(mathutil.Max(resultTp.GetFlen(), childTpCharLen*childTp.GetFlen()))
}
Expand Down Expand Up @@ -1999,7 +2005,7 @@ CheckReferenced:
// getUintFromNode gets uint64 value from ast.Node.
// For ordinary statement, node should be uint64 constant value.
// For prepared statement, node is string. We should convert it to uint64.
func getUintFromNode(ctx sessionctx.Context, n ast.Node) (uVal uint64, isNull bool, isExpectedType bool) {
func getUintFromNode(ctx sessionctx.Context, n ast.Node, mustInt64orUint64 bool) (uVal uint64, isNull bool, isExpectedType bool) {
var val interface{}
switch v := n.(type) {
case *driver.ValueExpr:
Expand All @@ -2008,6 +2014,11 @@ func getUintFromNode(ctx sessionctx.Context, n ast.Node) (uVal uint64, isNull bo
if !v.InExecute {
return 0, false, true
}
if mustInt64orUint64 {
if expected := checkParamTypeInt64orUint64(v); !expected {
return 0, false, false
}
}
param, err := expression.ParamMarkerExpression(ctx, v, false)
if err != nil {
return 0, false, false
Expand Down Expand Up @@ -2041,17 +2052,32 @@ func getUintFromNode(ctx sessionctx.Context, n ast.Node) (uVal uint64, isNull bo
return 0, false, false
}

// check param type for plan cache limit, only allow int64 and uint64 now
// eg: set @a = 1;
func checkParamTypeInt64orUint64(param *driver.ParamMarkerExpr) bool {
val := param.GetValue()
switch v := val.(type) {
case int64:
if v >= 0 {
return true
}
case uint64:
return true
}
return false
}

func extractLimitCountOffset(ctx sessionctx.Context, limit *ast.Limit) (count uint64,
offset uint64, err error) {
var isExpectedType bool
if limit.Count != nil {
count, _, isExpectedType = getUintFromNode(ctx, limit.Count)
count, _, isExpectedType = getUintFromNode(ctx, limit.Count, true)
if !isExpectedType {
return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT")
}
}
if limit.Offset != nil {
offset, _, isExpectedType = getUintFromNode(ctx, limit.Offset)
offset, _, isExpectedType = getUintFromNode(ctx, limit.Offset, true)
if !isExpectedType {
return 0, 0, ErrWrongArguments.GenWithStackByArgs("LIMIT")
}
Expand Down Expand Up @@ -2832,7 +2858,7 @@ func (g *gbyResolver) Enter(inNode ast.Node) (ast.Node, bool) {
case *driver.ParamMarkerExpr:
g.isParam = true
if g.exprDepth == 1 {
_, isNull, isExpectedType := getUintFromNode(g.ctx, n)
_, isNull, isExpectedType := getUintFromNode(g.ctx, n, false)
// For constant uint expression in top level, it should be treated as position expression.
if !isNull && isExpectedType {
return expression.ConstructPositionExpr(n), true
Expand Down Expand Up @@ -3608,7 +3634,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
// Set warning for the hint that requires the table name.
switch hint.HintName.L {
case TiDBMergeJoin, HintSMJ, TiDBIndexNestedLoopJoin, HintINLJ, HintINLHJ, HintINLMJ,
TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintIndexMerge, HintLeading:
TiDBHashJoin, HintHJ, HintUseIndex, HintIgnoreIndex, HintForceIndex, HintKeepOrder, HintNoKeepOrder, HintIndexMerge, HintLeading:
if len(hint.Tables) == 0 {
b.pushHintWithoutTableWarning(hint)
continue
Expand Down Expand Up @@ -3644,48 +3670,31 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev
aggHints.preferAggType |= preferStreamAgg
case HintAggToCop:
aggHints.preferAggToCop = true
case HintUseIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintUse,
HintScope: ast.HintForScan,
},
})
case HintIgnoreIndex:
case HintUseIndex, HintIgnoreIndex, HintForceIndex, HintKeepOrder, HintNoKeepOrder:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintIgnore,
HintScope: ast.HintForScan,
},
})
case HintForceIndex:
dbName := hint.Tables[0].DBName
if dbName.L == "" {
dbName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB)
var hintType ast.IndexHintType
switch hint.HintName.L {
case HintUseIndex:
hintType = ast.HintUse
case HintIgnoreIndex:
hintType = ast.HintIgnore
case HintForceIndex:
hintType = ast.HintForce
case HintKeepOrder:
hintType = ast.HintKeepOrder
case HintNoKeepOrder:
hintType = ast.HintNoKeepOrder
}
indexHintList = append(indexHintList, indexHintInfo{
dbName: dbName,
tblName: hint.Tables[0].TableName,
partitions: hint.Tables[0].PartitionList,
indexHint: &ast.IndexHint{
IndexNames: hint.Indexes,
HintType: ast.HintForce,
HintType: hintType,
HintScope: ast.HintForScan,
},
})
Expand Down Expand Up @@ -6214,7 +6223,7 @@ func (b *PlanBuilder) buildWindowFunctionFrameBound(_ context.Context, spec *ast
if bound.Type == ast.CurrentRow {
return bound, nil
}
numRows, _, _ := getUintFromNode(b.ctx, boundClause.Expr)
numRows, _, _ := getUintFromNode(b.ctx, boundClause.Expr, false)
bound.Num = numRows
return bound, nil
}
Expand Down Expand Up @@ -6530,7 +6539,7 @@ func (b *PlanBuilder) checkOriginWindowFrameBound(bound *ast.FrameBound, spec *a
if bound.Unit != ast.TimeUnitInvalid {
return ErrWindowRowsIntervalUse.GenWithStackByArgs(getWindowName(spec.Name.O))
}
_, isNull, isExpectedType := getUintFromNode(b.ctx, bound.Expr)
_, isNull, isExpectedType := getUintFromNode(b.ctx, bound.Expr, false)
if isNull || !isExpectedType {
return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O))
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/plan_cost_ver2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,7 @@ func TestCostModelVer2ScanRowSize(t *testing.T) {
tk.MustExec(`create table t (pk int, a int, b int, c int, d int, primary key(pk), index ab(a, b), index abc(a, b, c))`)
tk.MustExec("insert into t values (1, 1, 1, 1, 1)")
tk.MustExec(`set @@tidb_cost_model_version=2`)
tk.MustExec("set global tidb_enable_collect_execution_info=1;")

cases := []struct {
query string
Expand Down
6 changes: 6 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1421,6 +1421,12 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i
// our cost estimation is not reliable.
hasUseOrForce = true
path.Forced = true
if hint.HintType == ast.HintKeepOrder {
path.ForceKeepOrder = true
}
if hint.HintType == ast.HintNoKeepOrder {
path.ForceNoKeepOrder = true
}
available = append(available, path)
}
}
Expand Down
11 changes: 11 additions & 0 deletions planner/core/testdata/integration_suite_in.json
Original file line number Diff line number Diff line change
Expand Up @@ -653,6 +653,17 @@
"desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt"
]
},
{
"name": "TestKeepOrderHint",
"cases": [
"explain select /*+ keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"explain select /*+ keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t, primary) */ * from t where a<10 order by a limit 1;",
"explain select /*+ no_keep_order(t1, idx_a) */ * from t1 where a<10 limit 1;",
"explain select /*+ no_keep_order(t, primary) */ * from t where a<10 limit 1;"
]
},
{
"name": "TestViewHint",
"cases": [
Expand Down
Loading

0 comments on commit fb81453

Please sign in to comment.