diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 59c511249de8e..dcf317fa8f6e6 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -878,7 +878,7 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou if len(is.filterCondition) > 0 { var indexConds, tableConds []expression.Expression if copTask.tablePlan != nil { - indexConds, tableConds = ranger.DetachIndexFilterConditions(is.filterCondition, is.Index.Columns, is.Table) + indexConds, tableConds = splitIndexFilterConditions(is.filterCondition, is.Index.Columns, is.Table) } else { indexConds = is.filterCondition } @@ -921,6 +921,49 @@ func matchIndicesProp(idxCols []*model.IndexColumn, propCols []*expression.Colum return true } +func splitIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, + table *model.TableInfo) (indexConds, tableConds []expression.Expression) { + var pkName model.CIStr + if table.PKIsHandle { + for _, colInfo := range table.Columns { + if mysql.HasPriKeyFlag(colInfo.Flag) { + pkName = colInfo.Name + break + } + } + } + var indexConditions, tableConditions []expression.Expression + for _, cond := range conditions { + if checkIndexCondition(cond, indexColumns, pkName) { + indexConditions = append(indexConditions, cond) + } else { + tableConditions = append(tableConditions, cond) + } + } + return indexConditions, tableConditions +} + +// checkIndexCondition will check whether all columns of condition is index columns or primary key column. +func checkIndexCondition(condition expression.Expression, indexColumns []*model.IndexColumn, pkName model.CIStr) bool { + cols := expression.ExtractColumns(condition) + for _, col := range cols { + if pkName.L == col.ColName.L { + continue + } + isIndexColumn := false + for _, indCol := range indexColumns { + if col.ColName.L == indCol.Name.L && indCol.Length == types.UnspecifiedLength { + isIndexColumn = true + break + } + } + if !isIndexColumn { + return false + } + } + return true +} + func (p *DataSource) forceToTableScan() PhysicalPlan { ts := PhysicalTableScan{ Table: p.tableInfo, diff --git a/util/ranger/refiner.go b/util/ranger/checker.go similarity index 60% rename from util/ranger/refiner.go rename to util/ranger/checker.go index 994fa1a143e06..37432cc1b8288 100644 --- a/util/ranger/refiner.go +++ b/util/ranger/checker.go @@ -14,33 +14,12 @@ package ranger import ( - "math" - - "github.com/juju/errors" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" ) -// fullRange is (-∞, +∞). -var fullRange = []point{ - {start: true}, - {value: types.MaxValueDatum()}, -} - -// FullIntRange is (-∞, +∞) for IntColumnRange. -func FullIntRange() []types.IntColumnRange { - return []types.IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}} -} - -// FullIndexRange is (-∞, +∞) for IndexRange. -func FullIndexRange() []*types.IndexRange { - return []*types.IndexRange{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}} -} - // getEQFunctionOffset judge if the expression is a eq function like A = 1 where a is an index. // If so, it will return the offset of A in index columns. e.g. for index(C,B,A), A's offset is 2. func getEQFunctionOffset(expr expression.Expression, cols []*model.IndexColumn) int { @@ -68,110 +47,6 @@ func getEQFunctionOffset(expr expression.Expression, cols []*model.IndexColumn) return -1 } -func removeAccessConditions(conditions, accessConds []expression.Expression) []expression.Expression { - for i := len(conditions) - 1; i >= 0; i-- { - for _, cond := range accessConds { - if cond == conditions[i] { - conditions = append(conditions[:i], conditions[i+1:]...) - break - } - } - } - return conditions -} - -// checkIndexCondition will check whether all columns of condition is index columns or primary key column. -func checkIndexCondition(condition expression.Expression, indexColumns []*model.IndexColumn, pKName model.CIStr) bool { - cols := expression.ExtractColumns(condition) - for _, col := range cols { - if pKName.L == col.ColName.L { - continue - } - isIndexColumn := false - for _, indCol := range indexColumns { - if col.ColName.L == indCol.Name.L && indCol.Length == types.UnspecifiedLength { - isIndexColumn = true - break - } - } - if !isIndexColumn { - return false - } - } - return true -} - -// DetachIndexFilterConditions will detach the access conditions from other conditions. -func DetachIndexFilterConditions(conditions []expression.Expression, indexColumns []*model.IndexColumn, table *model.TableInfo) ([]expression.Expression, []expression.Expression) { - var pKName model.CIStr - if table.PKIsHandle { - for _, colInfo := range table.Columns { - if mysql.HasPriKeyFlag(colInfo.Flag) { - pKName = colInfo.Name - break - } - } - } - var indexConditions, tableConditions []expression.Expression - for _, cond := range conditions { - if checkIndexCondition(cond, indexColumns, pKName) { - indexConditions = append(indexConditions, cond) - } else { - tableConditions = append(tableConditions, cond) - } - } - return indexConditions, tableConditions -} - -// DetachColumnConditions distinguishes between access conditions and filter conditions from conditions. -func DetachColumnConditions(conditions []expression.Expression, colName model.CIStr) ([]expression.Expression, []expression.Expression) { - if colName.L == "" { - return nil, conditions - } - - var accessConditions, filterConditions []expression.Expression - checker := conditionChecker{ - colName: colName, - length: types.UnspecifiedLength, - } - for _, cond := range conditions { - cond = expression.PushDownNot(cond, false, nil) - if !checker.check(cond) { - filterConditions = append(filterConditions, cond) - continue - } - accessConditions = append(accessConditions, cond) - // TODO: it will lead to repeated computation cost. - if checker.shouldReserve { - filterConditions = append(filterConditions, cond) - checker.shouldReserve = false - } - } - - return accessConditions, filterConditions -} - -// BuildTableRange will build range of pk for PhysicalTableScan -func BuildTableRange(accessConditions []expression.Expression, sc *variable.StatementContext) ([]types.IntColumnRange, error) { - if len(accessConditions) == 0 { - return FullIntRange(), nil - } - - rb := builder{sc: sc} - rangePoints := fullRange - for _, cond := range accessConditions { - rangePoints = rb.intersection(rangePoints, rb.build(cond)) - if rb.err != nil { - return nil, errors.Trace(rb.err) - } - } - ranges, err := points2TableRanges(sc, rangePoints) - if err != nil { - return nil, errors.Trace(rb.err) - } - return ranges, nil -} - // conditionChecker checks if this condition can be pushed to index plan. type conditionChecker struct { idx *model.IndexInfo @@ -284,6 +159,7 @@ func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool pattern, ok := scalar.GetArgs()[1].(*expression.Constant) if !ok { return false + } if pattern.Value.IsNull() { return false diff --git a/util/ranger/new_refiner.go b/util/ranger/detacher.go similarity index 50% rename from util/ranger/new_refiner.go rename to util/ranger/detacher.go index 2523102f3f3a6..3ab3e0953ef81 100644 --- a/util/ranger/new_refiner.go +++ b/util/ranger/detacher.go @@ -14,135 +14,13 @@ package ranger import ( - "github.com/juju/errors" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" ) -func buildIndexRange(sc *variable.StatementContext, cols []*expression.Column, lengths []int, - accessCondition []expression.Expression) ([]*types.IndexRange, error) { - rb := builder{sc: sc} - var ( - ranges []*types.IndexRange - eqAndInCount int - err error - ) - for eqAndInCount = 0; eqAndInCount < len(accessCondition) && eqAndInCount < len(cols); eqAndInCount++ { - if sf, ok := accessCondition[eqAndInCount].(*expression.ScalarFunction); !ok || (sf.FuncName.L != ast.EQ && sf.FuncName.L != ast.In) { - break - } - // Build ranges for equal or in access conditions. - point := rb.build(accessCondition[eqAndInCount]) - if rb.err != nil { - return nil, errors.Trace(rb.err) - } - if eqAndInCount == 0 { - ranges, err = points2IndexRanges(sc, point, cols[eqAndInCount].RetType) - } else { - ranges, err = appendPoints2IndexRanges(sc, ranges, point, cols[eqAndInCount].RetType) - } - if err != nil { - return nil, errors.Trace(err) - } - } - rangePoints := fullRange - // Build rangePoints for non-equal access conditions. - for i := eqAndInCount; i < len(accessCondition); i++ { - rangePoints = rb.intersection(rangePoints, rb.build(accessCondition[i])) - if rb.err != nil { - return nil, errors.Trace(err) - } - } - if eqAndInCount == 0 { - ranges, err = points2IndexRanges(sc, rangePoints, cols[0].RetType) - } else if eqAndInCount < len(accessCondition) { - ranges, err = appendPoints2IndexRanges(sc, ranges, rangePoints, cols[eqAndInCount].RetType) - } - - // Take prefix index into consideration. - if hasPrefix(lengths) { - fixPrefixColRange(ranges, lengths) - } - - if len(ranges) > 0 && len(ranges[0].LowVal) < len(cols) { - for _, ran := range ranges { - if ran.HighExclude || ran.LowExclude { - if ran.HighExclude { - ran.HighVal = append(ran.HighVal, types.NewDatum(nil)) - } else { - ran.HighVal = append(ran.HighVal, types.MaxValueDatum()) - } - if ran.LowExclude { - ran.LowVal = append(ran.LowVal, types.MaxValueDatum()) - } else { - ran.LowVal = append(ran.LowVal, types.NewDatum(nil)) - } - } - } - } - return ranges, nil -} - -func hasPrefix(lengths []int) bool { - for _, l := range lengths { - if l != types.UnspecifiedLength { - return true - } - } - return false -} - -func fixPrefixColRange(ranges []*types.IndexRange, lengths []int) { - for _, ran := range ranges { - for i := 0; i < len(ran.LowVal); i++ { - fixRangeDatum(&ran.LowVal[i], lengths[i]) - } - ran.LowExclude = false - for i := 0; i < len(ran.HighVal); i++ { - fixRangeDatum(&ran.HighVal[i], lengths[i]) - } - ran.HighExclude = false - } -} - -func fixRangeDatum(v *types.Datum, length int) { - // If this column is prefix and the prefix length is smaller than the range, cut it. - if length != types.UnspecifiedLength && length < len(v.GetBytes()) { - v.SetBytes(v.GetBytes()[:length]) - } -} - -// getEQColOffset judge if the expression is a eq function that one side is constant and another is column. -// If so, it will return the offset of this column in the slice. -func getEQColOffset(expr expression.Expression, cols []*expression.Column) int { - f, ok := expr.(*expression.ScalarFunction) - if !ok || f.FuncName.L != ast.EQ { - return -1 - } - if c, ok := f.GetArgs()[0].(*expression.Column); ok { - if _, ok := f.GetArgs()[1].(*expression.Constant); ok { - for i, col := range cols { - if col.Equal(c, nil) { - return i - } - } - } - } - if c, ok := f.GetArgs()[1].(*expression.Column); ok { - if _, ok := f.GetArgs()[0].(*expression.Constant); ok { - for i, col := range cols { - if col.Equal(c, nil) { - return i - } - } - } - } - return -1 -} - // detachColumnCNFConditions detaches the condition for calculating range from the other conditions. // Please make sure that the top level is CNF form. func detachColumnCNFConditions(conditions []expression.Expression, checker *conditionChecker) ([]expression.Expression, []expression.Expression) { @@ -266,38 +144,57 @@ func DetachIndexConditions(conditions []expression.Expression, cols []*expressio return accessConds, filterConds } -// buildColumnRange builds the range for sampling histogram to calculate the row count. -func buildColumnRange(conds []expression.Expression, sc *variable.StatementContext, tp *types.FieldType) ([]*types.ColumnRange, error) { - if len(conds) == 0 { - return []*types.ColumnRange{{Low: types.Datum{}, High: types.MaxValueDatum()}}, nil - } - - rb := builder{sc: sc} - rangePoints := fullRange - for _, cond := range conds { - rangePoints = rb.intersection(rangePoints, rb.build(cond)) - if rb.err != nil { - return nil, errors.Trace(rb.err) +func removeAccessConditions(conditions, accessConds []expression.Expression) []expression.Expression { + for i := len(conditions) - 1; i >= 0; i-- { + for _, cond := range accessConds { + if cond == conditions[i] { + conditions = append(conditions[:i], conditions[i+1:]...) + break + } } } - ranges, err := points2ColumnRanges(sc, rangePoints, tp) - if err != nil { - return nil, errors.Trace(rb.err) - } - return ranges, nil + return conditions } // DetachCondsForSelectivity detaches the conditions used for range calculation from other useless conditions. func DetachCondsForSelectivity(conds []expression.Expression, rangeType int, cols []*expression.Column, lengths []int) (accessConditions, otherConditions []expression.Expression) { if rangeType == IntRangeType || rangeType == ColumnRangeType { - return DetachColumnConditions(conds, cols[0].ColName) + return detachColumnConditions(conds, cols[0].ColName) } else if rangeType == IndexRangeType { return DetachIndexConditions(conds, cols, lengths) } return nil, conds } +// detachColumnConditions distinguishes between access conditions and filter conditions from conditions. +func detachColumnConditions(conditions []expression.Expression, colName model.CIStr) ([]expression.Expression, []expression.Expression) { + if colName.L == "" { + return nil, conditions + } + + var accessConditions, filterConditions []expression.Expression + checker := conditionChecker{ + colName: colName, + length: types.UnspecifiedLength, + } + for _, cond := range conditions { + cond = expression.PushDownNot(cond, false, nil) + if !checker.check(cond) { + filterConditions = append(filterConditions, cond) + continue + } + accessConditions = append(accessConditions, cond) + // TODO: it will lead to repeated computation cost. + if checker.shouldReserve { + filterConditions = append(filterConditions, cond) + checker.shouldReserve = false + } + } + + return accessConditions, filterConditions +} + // DetachCondsForTableRange detaches the conditions used for range calculation form other useless conditions for // calculating the table range. func DetachCondsForTableRange(ctx context.Context, conds []expression.Expression, col *expression.Column) (accessContditions, otherConditions []expression.Expression) { @@ -310,64 +207,3 @@ func DetachCondsForTableRange(ctx context.Context, conds []expression.Expression } return detachColumnCNFConditions(conds, checker) } - -// BuildRange is a method which can calculate IntColumnRange, ColumnRange, IndexRange. -func BuildRange(sc *variable.StatementContext, conds []expression.Expression, rangeType int, cols []*expression.Column, - lengths []int) (retRanges []types.Range, _ error) { - if rangeType == IntRangeType { - ranges, err := BuildTableRange(conds, sc) - if err != nil { - return nil, errors.Trace(err) - } - retRanges = make([]types.Range, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran) - } - } else if rangeType == ColumnRangeType { - ranges, err := buildColumnRange(conds, sc, cols[0].RetType) - if err != nil { - return nil, errors.Trace(err) - } - retRanges = make([]types.Range, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran) - } - } else if rangeType == IndexRangeType { - ranges, err := buildIndexRange(sc, cols, lengths, conds) - if err != nil { - return nil, errors.Trace(err) - } - retRanges = make([]types.Range, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran) - } - } - return -} - -// Ranges2IntRanges changes []types.Range to []types.IntColumnRange -func Ranges2IntRanges(ranges []types.Range) []types.IntColumnRange { - retRanges := make([]types.IntColumnRange, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran.Convert2IntRange()) - } - return retRanges -} - -// Ranges2ColumnRanges changes []types.Range to []*types.ColumnRange -func Ranges2ColumnRanges(ranges []types.Range) []*types.ColumnRange { - retRanges := make([]*types.ColumnRange, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran.Convert2ColumnRange()) - } - return retRanges -} - -// Ranges2IndexRanges changes []types.Range to []*types.IndexRange -func Ranges2IndexRanges(ranges []types.Range) []*types.IndexRange { - retRanges := make([]*types.IndexRange, 0, len(ranges)) - for _, ran := range ranges { - retRanges = append(retRanges, ran.Convert2IndexRange()) - } - return retRanges -} diff --git a/util/ranger/range.go b/util/ranger/points.go similarity index 67% rename from util/ranger/range.go rename to util/ranger/points.go index 439896f4f2ff2..d7ab1e2e5ff96 100644 --- a/util/ranger/range.go +++ b/util/ranger/points.go @@ -114,6 +114,22 @@ func (r *pointSorter) Swap(i, j int) { r.points[i], r.points[j] = r.points[j], r.points[i] } +// fullRange is (-∞, +∞). +var fullRange = []point{ + {start: true}, + {value: types.MaxValueDatum()}, +} + +// FullIntRange is (-∞, +∞) for IntColumnRange. +func FullIntRange() []types.IntColumnRange { + return []types.IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}} +} + +// FullIndexRange is (-∞, +∞) for IndexRange. +func FullIndexRange() []*types.IndexRange { + return []*types.IndexRange{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}} +} + // builder is the range builder struct. type builder struct { err error @@ -487,216 +503,3 @@ func (r *builder) merge(a, b []point, union bool) []point { } return merged } - -// points2IndexRanges build index ranges from range points. -// Only the first column in the index is built, extra column ranges will be appended by -// appendPoints2IndexRanges. -func points2IndexRanges(sc *variable.StatementContext, rangePoints []point, tp *types.FieldType) ([]*types.IndexRange, error) { - indexRanges := make([]*types.IndexRange, 0, len(rangePoints)/2) - for i := 0; i < len(rangePoints); i += 2 { - startPoint, err := convertPoint(sc, rangePoints[i], tp) - if err != nil { - return nil, errors.Trace(err) - } - endPoint, err := convertPoint(sc, rangePoints[i+1], tp) - if err != nil { - return nil, errors.Trace(err) - } - less, err := rangePointLess(sc, startPoint, endPoint) - if err != nil { - return nil, errors.Trace(err) - } - if !less { - continue - } - ir := &types.IndexRange{ - LowVal: []types.Datum{startPoint.value}, - LowExclude: startPoint.excl, - HighVal: []types.Datum{endPoint.value}, - HighExclude: endPoint.excl, - } - indexRanges = append(indexRanges, ir) - } - return indexRanges, nil -} - -func convertPoint(sc *variable.StatementContext, point point, tp *types.FieldType) (point, error) { - switch point.value.Kind() { - case types.KindMaxValue, types.KindMinNotNull: - return point, nil - } - casted, err := point.value.ConvertTo(sc, tp) - if err != nil { - return point, errors.Trace(err) - } - valCmpCasted, err := point.value.CompareDatum(sc, &casted) - if err != nil { - return point, errors.Trace(err) - } - point.value = casted - if valCmpCasted == 0 { - return point, nil - } - if point.start { - if point.excl { - if valCmpCasted < 0 { - // e.g. "a > 1.9" convert to "a >= 2". - point.excl = false - } - } else { - if valCmpCasted > 0 { - // e.g. "a >= 1.1 convert to "a > 1" - point.excl = true - } - } - } else { - if point.excl { - if valCmpCasted > 0 { - // e.g. "a < 1.1" convert to "a <= 1" - point.excl = false - } - } else { - if valCmpCasted < 0 { - // e.g. "a <= 1.9" convert to "a < 2" - point.excl = true - } - } - } - return point, nil -} - -// appendPoints2IndexRanges appends additional column ranges for multi-column index. -// The additional column ranges can only be appended to point ranges. -// for example we have an index (a, b), if the condition is (a > 1 and b = 2) -// then we can not build a conjunctive ranges for this index. -func appendPoints2IndexRanges(sc *variable.StatementContext, origin []*types.IndexRange, rangePoints []point, - ft *types.FieldType) ([]*types.IndexRange, error) { - var newIndexRanges []*types.IndexRange - for i := 0; i < len(origin); i++ { - oRange := origin[i] - if !oRange.IsPoint(sc) { - newIndexRanges = append(newIndexRanges, oRange) - } else { - newRanges, err := appendPoints2IndexRange(sc, oRange, rangePoints, ft) - if err != nil { - return nil, errors.Trace(err) - } - newIndexRanges = append(newIndexRanges, newRanges...) - } - } - return newIndexRanges, nil -} - -func appendPoints2IndexRange(sc *variable.StatementContext, origin *types.IndexRange, rangePoints []point, - ft *types.FieldType) ([]*types.IndexRange, error) { - newRanges := make([]*types.IndexRange, 0, len(rangePoints)/2) - for i := 0; i < len(rangePoints); i += 2 { - startPoint, err := convertPoint(sc, rangePoints[i], ft) - if err != nil { - return nil, errors.Trace(err) - } - endPoint, err := convertPoint(sc, rangePoints[i+1], ft) - if err != nil { - return nil, errors.Trace(err) - } - less, err := rangePointLess(sc, startPoint, endPoint) - if err != nil { - return nil, errors.Trace(err) - } - if !less { - continue - } - - lowVal := make([]types.Datum, len(origin.LowVal)+1) - copy(lowVal, origin.LowVal) - lowVal[len(origin.LowVal)] = startPoint.value - - highVal := make([]types.Datum, len(origin.HighVal)+1) - copy(highVal, origin.HighVal) - highVal[len(origin.HighVal)] = endPoint.value - - ir := &types.IndexRange{ - LowVal: lowVal, - LowExclude: startPoint.excl, - HighVal: highVal, - HighExclude: endPoint.excl, - } - newRanges = append(newRanges, ir) - } - return newRanges, nil -} - -// points2TableRanges will construct the range slice with the given range points -func points2TableRanges(sc *variable.StatementContext, rangePoints []point) ([]types.IntColumnRange, error) { - tableRanges := make([]types.IntColumnRange, 0, len(rangePoints)/2) - for i := 0; i < len(rangePoints); i += 2 { - startPoint := rangePoints[i] - if startPoint.value.IsNull() || startPoint.value.Kind() == types.KindMinNotNull { - startPoint.value.SetInt64(math.MinInt64) - } - startInt, err := startPoint.value.ToInt64(sc) - if err != nil { - return nil, errors.Trace(err) - } - startDatum := types.NewDatum(startInt) - cmp, err := startDatum.CompareDatum(sc, &startPoint.value) - if err != nil { - return nil, errors.Trace(err) - } - if cmp < 0 || (cmp == 0 && startPoint.excl) { - startInt++ - } - endPoint := rangePoints[i+1] - if endPoint.value.IsNull() { - endPoint.value.SetInt64(math.MinInt64) - } else if endPoint.value.Kind() == types.KindMaxValue { - endPoint.value.SetInt64(math.MaxInt64) - } - endInt, err := endPoint.value.ToInt64(sc) - if err != nil { - return nil, errors.Trace(err) - } - endDatum := types.NewDatum(endInt) - cmp, err = endDatum.CompareDatum(sc, &endPoint.value) - if err != nil { - return nil, errors.Trace(err) - } - if cmp > 0 || (cmp == 0 && endPoint.excl) { - endInt-- - } - if startInt > endInt { - continue - } - tableRanges = append(tableRanges, types.IntColumnRange{LowVal: startInt, HighVal: endInt}) - } - return tableRanges, nil -} - -func points2ColumnRanges(sc *variable.StatementContext, points []point, tp *types.FieldType) ([]*types.ColumnRange, error) { - columnRanges := make([]*types.ColumnRange, 0, len(points)/2) - for i := 0; i < len(points); i += 2 { - startPoint, err := convertPoint(sc, points[i], tp) - if err != nil { - return nil, errors.Trace(err) - } - endPoint, err := convertPoint(sc, points[i+1], tp) - if err != nil { - return nil, errors.Trace(err) - } - less, err := rangePointLess(sc, startPoint, endPoint) - if err != nil { - return nil, errors.Trace(err) - } - if !less { - continue - } - cr := &types.ColumnRange{ - Low: startPoint.value, - LowExcl: startPoint.excl, - High: endPoint.value, - HighExcl: endPoint.excl, - } - columnRanges = append(columnRanges, cr) - } - return columnRanges, nil -} diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go new file mode 100644 index 0000000000000..ce505aa8af770 --- /dev/null +++ b/util/ranger/ranger.go @@ -0,0 +1,464 @@ +// Copyright 2017 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 ranger + +import ( + "math" + + "github.com/juju/errors" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" +) + +// points2IndexRanges build index ranges from range points. +// Only the first column in the index is built, extra column ranges will be appended by +// appendPoints2IndexRanges. +func points2IndexRanges(sc *variable.StatementContext, rangePoints []point, tp *types.FieldType) ([]*types.IndexRange, error) { + indexRanges := make([]*types.IndexRange, 0, len(rangePoints)/2) + for i := 0; i < len(rangePoints); i += 2 { + startPoint, err := convertPoint(sc, rangePoints[i], tp) + if err != nil { + return nil, errors.Trace(err) + } + endPoint, err := convertPoint(sc, rangePoints[i+1], tp) + if err != nil { + return nil, errors.Trace(err) + } + less, err := rangePointLess(sc, startPoint, endPoint) + if err != nil { + return nil, errors.Trace(err) + } + if !less { + continue + } + ir := &types.IndexRange{ + LowVal: []types.Datum{startPoint.value}, + LowExclude: startPoint.excl, + HighVal: []types.Datum{endPoint.value}, + HighExclude: endPoint.excl, + } + indexRanges = append(indexRanges, ir) + } + return indexRanges, nil +} + +func convertPoint(sc *variable.StatementContext, point point, tp *types.FieldType) (point, error) { + switch point.value.Kind() { + case types.KindMaxValue, types.KindMinNotNull: + return point, nil + } + casted, err := point.value.ConvertTo(sc, tp) + if err != nil { + return point, errors.Trace(err) + } + valCmpCasted, err := point.value.CompareDatum(sc, &casted) + if err != nil { + return point, errors.Trace(err) + } + point.value = casted + if valCmpCasted == 0 { + return point, nil + } + if point.start { + if point.excl { + if valCmpCasted < 0 { + // e.g. "a > 1.9" convert to "a >= 2". + point.excl = false + } + } else { + if valCmpCasted > 0 { + // e.g. "a >= 1.1 convert to "a > 1" + point.excl = true + } + } + } else { + if point.excl { + if valCmpCasted > 0 { + // e.g. "a < 1.1" convert to "a <= 1" + point.excl = false + } + } else { + if valCmpCasted < 0 { + // e.g. "a <= 1.9" convert to "a < 2" + point.excl = true + } + } + } + return point, nil +} + +// appendPoints2IndexRanges appends additional column ranges for multi-column index. +// The additional column ranges can only be appended to point ranges. +// for example we have an index (a, b), if the condition is (a > 1 and b = 2) +// then we can not build a conjunctive ranges for this index. +func appendPoints2IndexRanges(sc *variable.StatementContext, origin []*types.IndexRange, rangePoints []point, + ft *types.FieldType) ([]*types.IndexRange, error) { + var newIndexRanges []*types.IndexRange + for i := 0; i < len(origin); i++ { + oRange := origin[i] + if !oRange.IsPoint(sc) { + newIndexRanges = append(newIndexRanges, oRange) + } else { + newRanges, err := appendPoints2IndexRange(sc, oRange, rangePoints, ft) + if err != nil { + return nil, errors.Trace(err) + } + newIndexRanges = append(newIndexRanges, newRanges...) + } + } + return newIndexRanges, nil +} + +func appendPoints2IndexRange(sc *variable.StatementContext, origin *types.IndexRange, rangePoints []point, + ft *types.FieldType) ([]*types.IndexRange, error) { + newRanges := make([]*types.IndexRange, 0, len(rangePoints)/2) + for i := 0; i < len(rangePoints); i += 2 { + startPoint, err := convertPoint(sc, rangePoints[i], ft) + if err != nil { + return nil, errors.Trace(err) + } + endPoint, err := convertPoint(sc, rangePoints[i+1], ft) + if err != nil { + return nil, errors.Trace(err) + } + less, err := rangePointLess(sc, startPoint, endPoint) + if err != nil { + return nil, errors.Trace(err) + } + if !less { + continue + } + + lowVal := make([]types.Datum, len(origin.LowVal)+1) + copy(lowVal, origin.LowVal) + lowVal[len(origin.LowVal)] = startPoint.value + + highVal := make([]types.Datum, len(origin.HighVal)+1) + copy(highVal, origin.HighVal) + highVal[len(origin.HighVal)] = endPoint.value + + ir := &types.IndexRange{ + LowVal: lowVal, + LowExclude: startPoint.excl, + HighVal: highVal, + HighExclude: endPoint.excl, + } + newRanges = append(newRanges, ir) + } + return newRanges, nil +} + +// points2TableRanges will construct the range slice with the given range points +func points2TableRanges(sc *variable.StatementContext, rangePoints []point) ([]types.IntColumnRange, error) { + tableRanges := make([]types.IntColumnRange, 0, len(rangePoints)/2) + for i := 0; i < len(rangePoints); i += 2 { + startPoint := rangePoints[i] + if startPoint.value.IsNull() || startPoint.value.Kind() == types.KindMinNotNull { + startPoint.value.SetInt64(math.MinInt64) + } + startInt, err := startPoint.value.ToInt64(sc) + if err != nil { + return nil, errors.Trace(err) + } + startDatum := types.NewDatum(startInt) + cmp, err := startDatum.CompareDatum(sc, &startPoint.value) + if err != nil { + return nil, errors.Trace(err) + } + if cmp < 0 || (cmp == 0 && startPoint.excl) { + startInt++ + } + endPoint := rangePoints[i+1] + if endPoint.value.IsNull() { + endPoint.value.SetInt64(math.MinInt64) + } else if endPoint.value.Kind() == types.KindMaxValue { + endPoint.value.SetInt64(math.MaxInt64) + } + endInt, err := endPoint.value.ToInt64(sc) + if err != nil { + return nil, errors.Trace(err) + } + endDatum := types.NewDatum(endInt) + cmp, err = endDatum.CompareDatum(sc, &endPoint.value) + if err != nil { + return nil, errors.Trace(err) + } + if cmp > 0 || (cmp == 0 && endPoint.excl) { + endInt-- + } + if startInt > endInt { + continue + } + tableRanges = append(tableRanges, types.IntColumnRange{LowVal: startInt, HighVal: endInt}) + } + return tableRanges, nil +} + +func points2ColumnRanges(sc *variable.StatementContext, points []point, tp *types.FieldType) ([]*types.ColumnRange, error) { + columnRanges := make([]*types.ColumnRange, 0, len(points)/2) + for i := 0; i < len(points); i += 2 { + startPoint, err := convertPoint(sc, points[i], tp) + if err != nil { + return nil, errors.Trace(err) + } + endPoint, err := convertPoint(sc, points[i+1], tp) + if err != nil { + return nil, errors.Trace(err) + } + less, err := rangePointLess(sc, startPoint, endPoint) + if err != nil { + return nil, errors.Trace(err) + } + if !less { + continue + } + cr := &types.ColumnRange{ + Low: startPoint.value, + LowExcl: startPoint.excl, + High: endPoint.value, + HighExcl: endPoint.excl, + } + columnRanges = append(columnRanges, cr) + } + return columnRanges, nil +} + +// buildTableRange will build range of pk for PhysicalTableScan +func buildTableRange(accessConditions []expression.Expression, sc *variable.StatementContext) ([]types.IntColumnRange, error) { + if len(accessConditions) == 0 { + return FullIntRange(), nil + } + + rb := builder{sc: sc} + rangePoints := fullRange + for _, cond := range accessConditions { + rangePoints = rb.intersection(rangePoints, rb.build(cond)) + if rb.err != nil { + return nil, errors.Trace(rb.err) + } + } + ranges, err := points2TableRanges(sc, rangePoints) + if err != nil { + return nil, errors.Trace(err) + } + return ranges, nil +} + +// buildColumnRange builds the range for sampling histogram to calculate the row count. +func buildColumnRange(conds []expression.Expression, sc *variable.StatementContext, tp *types.FieldType) ([]*types.ColumnRange, error) { + if len(conds) == 0 { + return []*types.ColumnRange{{Low: types.Datum{}, High: types.MaxValueDatum()}}, nil + } + + rb := builder{sc: sc} + rangePoints := fullRange + for _, cond := range conds { + rangePoints = rb.intersection(rangePoints, rb.build(cond)) + if rb.err != nil { + return nil, errors.Trace(rb.err) + } + } + ranges, err := points2ColumnRanges(sc, rangePoints, tp) + if err != nil { + return nil, errors.Trace(err) + } + return ranges, nil +} + +func buildIndexRange(sc *variable.StatementContext, cols []*expression.Column, lengths []int, + accessCondition []expression.Expression) ([]*types.IndexRange, error) { + rb := builder{sc: sc} + var ( + ranges []*types.IndexRange + eqAndInCount int + err error + ) + for eqAndInCount = 0; eqAndInCount < len(accessCondition) && eqAndInCount < len(cols); eqAndInCount++ { + if sf, ok := accessCondition[eqAndInCount].(*expression.ScalarFunction); !ok || (sf.FuncName.L != ast.EQ && sf.FuncName.L != ast.In) { + break + } + // Build ranges for equal or in access conditions. + point := rb.build(accessCondition[eqAndInCount]) + if rb.err != nil { + return nil, errors.Trace(rb.err) + } + if eqAndInCount == 0 { + ranges, err = points2IndexRanges(sc, point, cols[eqAndInCount].RetType) + } else { + ranges, err = appendPoints2IndexRanges(sc, ranges, point, cols[eqAndInCount].RetType) + } + if err != nil { + return nil, errors.Trace(err) + } + } + rangePoints := fullRange + // Build rangePoints for non-equal access conditions. + for i := eqAndInCount; i < len(accessCondition); i++ { + rangePoints = rb.intersection(rangePoints, rb.build(accessCondition[i])) + if rb.err != nil { + return nil, errors.Trace(rb.err) + } + } + if eqAndInCount == 0 { + ranges, err = points2IndexRanges(sc, rangePoints, cols[0].RetType) + } else if eqAndInCount < len(accessCondition) { + ranges, err = appendPoints2IndexRanges(sc, ranges, rangePoints, cols[eqAndInCount].RetType) + } + if err != nil { + return nil, errors.Trace(err) + } + + // Take prefix index into consideration. + if hasPrefix(lengths) { + fixPrefixColRange(ranges, lengths) + } + + if len(ranges) > 0 && len(ranges[0].LowVal) < len(cols) { + for _, ran := range ranges { + if ran.HighExclude || ran.LowExclude { + if ran.HighExclude { + ran.HighVal = append(ran.HighVal, types.NewDatum(nil)) + } else { + ran.HighVal = append(ran.HighVal, types.MaxValueDatum()) + } + if ran.LowExclude { + ran.LowVal = append(ran.LowVal, types.MaxValueDatum()) + } else { + ran.LowVal = append(ran.LowVal, types.NewDatum(nil)) + } + } + } + } + return ranges, nil +} + +func hasPrefix(lengths []int) bool { + for _, l := range lengths { + if l != types.UnspecifiedLength { + return true + } + } + return false +} + +func fixPrefixColRange(ranges []*types.IndexRange, lengths []int) { + for _, ran := range ranges { + for i := 0; i < len(ran.LowVal); i++ { + fixRangeDatum(&ran.LowVal[i], lengths[i]) + } + ran.LowExclude = false + for i := 0; i < len(ran.HighVal); i++ { + fixRangeDatum(&ran.HighVal[i], lengths[i]) + } + ran.HighExclude = false + } +} + +func fixRangeDatum(v *types.Datum, length int) { + // If this column is prefix and the prefix length is smaller than the range, cut it. + if length != types.UnspecifiedLength && length < len(v.GetBytes()) { + v.SetBytes(v.GetBytes()[:length]) + } +} + +// getEQColOffset judge if the expression is a eq function that one side is constant and another is column. +// If so, it will return the offset of this column in the slice. +func getEQColOffset(expr expression.Expression, cols []*expression.Column) int { + f, ok := expr.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { + return -1 + } + if c, ok := f.GetArgs()[0].(*expression.Column); ok { + if _, ok := f.GetArgs()[1].(*expression.Constant); ok { + for i, col := range cols { + if col.Equal(c, nil) { + return i + } + } + } + } + if c, ok := f.GetArgs()[1].(*expression.Column); ok { + if _, ok := f.GetArgs()[0].(*expression.Constant); ok { + for i, col := range cols { + if col.Equal(c, nil) { + return i + } + } + } + } + return -1 +} + +// BuildRange is a method which can calculate IntColumnRange, ColumnRange, IndexRange. +func BuildRange(sc *variable.StatementContext, conds []expression.Expression, rangeType int, cols []*expression.Column, + lengths []int) (retRanges []types.Range, _ error) { + if rangeType == IntRangeType { + ranges, err := buildTableRange(conds, sc) + if err != nil { + return nil, errors.Trace(err) + } + retRanges = make([]types.Range, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran) + } + } else if rangeType == ColumnRangeType { + ranges, err := buildColumnRange(conds, sc, cols[0].RetType) + if err != nil { + return nil, errors.Trace(err) + } + retRanges = make([]types.Range, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran) + } + } else if rangeType == IndexRangeType { + ranges, err := buildIndexRange(sc, cols, lengths, conds) + if err != nil { + return nil, errors.Trace(err) + } + retRanges = make([]types.Range, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran) + } + } + return +} + +// Ranges2IntRanges changes []types.Range to []types.IntColumnRange +func Ranges2IntRanges(ranges []types.Range) []types.IntColumnRange { + retRanges := make([]types.IntColumnRange, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran.Convert2IntRange()) + } + return retRanges +} + +// Ranges2ColumnRanges changes []types.Range to []*types.ColumnRange +func Ranges2ColumnRanges(ranges []types.Range) []*types.ColumnRange { + retRanges := make([]*types.ColumnRange, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran.Convert2ColumnRange()) + } + return retRanges +} + +// Ranges2IndexRanges changes []types.Range to []*types.IndexRange +func Ranges2IndexRanges(ranges []types.Range) []*types.IndexRange { + retRanges := make([]*types.IndexRange, 0, len(ranges)) + for _, ran := range ranges { + retRanges = append(retRanges, ran.Convert2IndexRange()) + } + return retRanges +} diff --git a/util/ranger/range_test.go b/util/ranger/ranger_test.go similarity index 99% rename from util/ranger/range_test.go rename to util/ranger/ranger_test.go index 55a4e3088c57c..088ec0cf5c9e2 100644 --- a/util/ranger/range_test.go +++ b/util/ranger/ranger_test.go @@ -682,7 +682,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { col := expression.ColInfo2Col(sel.Schema().Columns, ds.TableInfo().Columns[tt.colPos]) c.Assert(col, NotNil) var filter []expression.Expression - conds, filter = ranger.DetachColumnConditions(conds, col.ColName) + conds, filter = ranger.DetachCondsForSelectivity(conds, ranger.ColumnRangeType, []*expression.Column{col}, nil) c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr)) c.Assert(fmt.Sprintf("%s", filter), Equals, tt.filterConds, Commentf("wrong filter conditions for expr: %s", tt.exprStr)) result, err := ranger.BuildRange(new(variable.StatementContext), conds, ranger.ColumnRangeType, []*expression.Column{col}, nil)