Skip to content

Commit

Permalink
plan: use Column.UniqueID in conditionChecker of ranger (#8236) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
XuHuaiyu authored and zz-jason committed Mar 7, 2019
1 parent 899b7fc commit bc42d7f
Show file tree
Hide file tree
Showing 6 changed files with 50 additions and 26 deletions.
2 changes: 1 addition & 1 deletion planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ func (s *partitionProcessor) canBePrune(ctx sessionctx.Context, col *expression.
conds = expression.PropagateConstant(ctx, conds)

// Calculate the column range to prune.
accessConds := ranger.ExtractAccessConditionsForColumn(conds, col.ColName)
accessConds := ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID)
r, err := ranger.BuildColumnRange(accessConds, ctx.GetSessionVars().StmtCtx, col.RetType)
if err != nil {
return false, errors.Trace(err)
Expand Down
2 changes: 1 addition & 1 deletion statistics/selectivity.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,7 +244,7 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran
var accessConds, remainedConds []expression.Expression
switch rangeType {
case ranger.ColumnRangeType:
accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].ColName)
accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].UniqueID)
ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType)
case ranger.IndexRangeType:
var res *ranger.DetachRangeResult
Expand Down
15 changes: 9 additions & 6 deletions table/tables/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,20 +115,23 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) {
partitionPruneExprs := make([]expression.Expression, 0, len(pi.Definitions))
locateExprs := make([]expression.Expression, 0, len(pi.Definitions))
var buf bytes.Buffer
dbName := model.NewCIStr(ctx.GetSessionVars().CurrentDB)
columns := expression.ColumnInfos2ColumnsWithDBName(ctx, dbName, tblInfo.Name, tblInfo.Columns)
schema := expression.NewSchema(columns...)
for i := 0; i < len(pi.Definitions); i++ {
if strings.EqualFold(pi.Definitions[i].LessThan[0], "MAXVALUE") {
// Expr less than maxvalue is always true.
fmt.Fprintf(&buf, "true")
} else {
fmt.Fprintf(&buf, "((%s) < (%s))", pi.Expr, pi.Definitions[i].LessThan[0])
}
expr, err := expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo)
exprs, err := expression.ParseSimpleExprsWithSchema(ctx, buf.String(), schema)
if err != nil {
// If it got an error here, ddl may hang forever, so this error log is important.
log.Error("wrong table partition expression:", errors.ErrorStack(err), buf.String())
return nil, errors.Trace(err)
}
locateExprs = append(locateExprs, expr)
locateExprs = append(locateExprs, exprs[0])

if i > 0 {
fmt.Fprintf(&buf, " and ((%s) >= (%s))", pi.Expr, pi.Definitions[i-1].LessThan[0])
Expand All @@ -137,8 +140,8 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) {
fmt.Fprintf(&buf, " or ((%s) is null)", pi.Expr)

// Extracts the column of the partition expression, it will be used by partition prunning.
if tmp, err1 := expression.ParseSimpleExprWithTableInfo(ctx, pi.Expr, tblInfo); err1 == nil {
if col, ok := tmp.(*expression.Column); ok {
if tmps, err1 := expression.ParseSimpleExprsWithSchema(ctx, pi.Expr, schema); err1 == nil {
if col, ok := tmps[0].(*expression.Column); ok {
column = col
}
}
Expand All @@ -147,13 +150,13 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) {
}
}

expr, err = expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo)
exprs, err = expression.ParseSimpleExprsWithSchema(ctx, buf.String(), schema)
if err != nil {
// If it got an error here, ddl may hang forever, so this error log is important.
log.Error("wrong table partition expression:", errors.ErrorStack(err), buf.String())
return nil, errors.Trace(err)
}
partitionPruneExprs = append(partitionPruneExprs, expr)
partitionPruneExprs = append(partitionPruneExprs, exprs[0])
buf.Reset()
}
return &PartitionExpr{
Expand Down
8 changes: 2 additions & 6 deletions util/ranger/checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,13 @@ package ranger

import (
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/types"
)

// conditionChecker checks if this condition can be pushed to index planner.
type conditionChecker struct {
colName model.CIStr
colUniqueID int64
shouldReserve bool // check if a access condition should be reserved in filter conditions.
length int
}
Expand Down Expand Up @@ -135,8 +134,5 @@ func (c *conditionChecker) checkColumn(expr expression.Expression) bool {
if !ok {
return false
}
if c.colName.L != "" {
return c.colName.L == col.ColName.L
}
return true
return c.colUniqueID == col.UniqueID
}
18 changes: 7 additions & 11 deletions util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package ranger
import (
"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -174,7 +173,7 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex
return res, nil
}
checker := &conditionChecker{
colName: cols[eqOrInCount].ColName,
colUniqueID: cols[eqOrInCount].UniqueID,
length: lengths[eqOrInCount],
shouldReserve: lengths[eqOrInCount] != types.UnspecifiedLength,
}
Expand Down Expand Up @@ -257,7 +256,7 @@ func detachDNFCondAndBuildRangeForIndex(sctx sessionctx.Context, condition *expr
cols []*expression.Column, newTpSlice []*types.FieldType, lengths []int) ([]*Range, []expression.Expression, bool, error) {
sc := sctx.GetSessionVars().StmtCtx
firstColumnChecker := &conditionChecker{
colName: cols[0].ColName,
colUniqueID: cols[0].UniqueID,
shouldReserve: lengths[0] != types.UnspecifiedLength,
length: lengths[0],
}
Expand Down Expand Up @@ -376,13 +375,10 @@ func removeAccessConditions(conditions, accessConds []expression.Expression) []e
}

// ExtractAccessConditionsForColumn detaches the access conditions used for range calculation.
func ExtractAccessConditionsForColumn(conds []expression.Expression, colName model.CIStr) []expression.Expression {
if colName.L == "" {
return nil
}
func ExtractAccessConditionsForColumn(conds []expression.Expression, uniqueID int64) []expression.Expression {
checker := conditionChecker{
colName: colName,
length: types.UnspecifiedLength,
colUniqueID: uniqueID,
length: types.UnspecifiedLength,
}
accessConds := make([]expression.Expression, 0, 8)
return expression.Filter(accessConds, conds, checker.check)
Expand All @@ -392,8 +388,8 @@ func ExtractAccessConditionsForColumn(conds []expression.Expression, colName mod
// calculating the table range.
func DetachCondsForTableRange(sctx sessionctx.Context, conds []expression.Expression, col *expression.Column) (accessContditions, otherConditions []expression.Expression) {
checker := &conditionChecker{
colName: col.ColName,
length: types.UnspecifiedLength,
colUniqueID: col.UniqueID,
length: types.UnspecifiedLength,
}
return detachColumnCNFConditions(sctx, conds, checker)
}
31 changes: 30 additions & 1 deletion util/ranger/ranger_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -954,11 +954,40 @@ func (s *testRangerSuite) TestColumnRange(c *C) {
}
col := expression.ColInfo2Col(sel.Schema().Columns, ds.TableInfo().Columns[tt.colPos])
c.Assert(col, NotNil)
conds = ranger.ExtractAccessConditionsForColumn(conds, col.ColName)
conds = ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID)
c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr))
result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType)
c.Assert(err, IsNil)
got := fmt.Sprintf("%v", result)
c.Assert(got, Equals, tt.resultStr, Commentf("different for expr %s, col: %v", tt.exprStr, col))
}
}

func (s *testRangerSuite) TestIndexRangeElimininatedProjection(c *C) {
defer testleak.AfterTest(c)()
dom, store, err := newDomainStoreWithBootstrap(c)
defer func() {
dom.Close()
store.Close()
}()
c.Assert(err, IsNil)
testKit := testkit.NewTestKit(c, store)
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int not null, b int not null, primary key(a,b))")
testKit.MustExec("insert into t values(1,2)")
testKit.MustExec("analyze table t")
testKit.MustQuery("explain select * from (select * from t union all select ifnull(a,b), b from t) sub where a > 0").Check(testkit.Rows(
"Union_11 1.80 root ",
"├─IndexReader_14 1.00 root index:IndexScan_13",
"│ └─IndexScan_13 1.00 cop table:t, index:a, b, range:(0 +inf,+inf +inf], keep order:false",
"└─Projection_16 0.80 root ifnull(test.t.a, test.t.b), test.t.b",
" └─TableReader_19 0.80 root data:Selection_18",
" └─Selection_18 0.80 cop gt(ifnull(test.t.a, test.t.b), 0)",
" └─TableScan_17 1.00 cop table:t, range:[-inf,+inf], keep order:false",
))
testKit.MustQuery("select * from (select * from t union all select ifnull(a,b), b from t) sub where a > 0").Check(testkit.Rows(
"1 2",
"1 2",
))
}

0 comments on commit bc42d7f

Please sign in to comment.