1 Star 0 Fork 0

zhoujin826/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
points.go 14.75 KB
一键复制 编辑 原始数据 按行查看 历史
zz-jason 提交于 2018-06-07 14:56 . plan: refactor errors (#6377) (#6777)
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542
// Copyright 2015 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 (
"fmt"
"math"
"sort"
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/types"
)
// Error instances.
var (
ErrUnsupportedType = terror.ClassOptimizer.New(CodeUnsupportedType, "Unsupported type")
)
// Error codes.
const (
CodeUnsupportedType terror.ErrCode = 1
)
// RangeType is alias for int.
type RangeType int
// RangeType constants.
const (
IntRangeType RangeType = iota
ColumnRangeType
IndexRangeType
)
// Point is the end point of range interval.
type point struct {
value types.Datum
excl bool // exclude
start bool
}
func (rp point) String() string {
val := rp.value.GetValue()
if rp.value.Kind() == types.KindMinNotNull {
val = "-inf"
} else if rp.value.Kind() == types.KindMaxValue {
val = "+inf"
}
if rp.start {
symbol := "["
if rp.excl {
symbol = "("
}
return fmt.Sprintf("%s%v", symbol, val)
}
symbol := "]"
if rp.excl {
symbol = ")"
}
return fmt.Sprintf("%v%s", val, symbol)
}
type pointSorter struct {
points []point
err error
sc *stmtctx.StatementContext
}
func (r *pointSorter) Len() int {
return len(r.points)
}
func (r *pointSorter) Less(i, j int) bool {
a := r.points[i]
b := r.points[j]
less, err := rangePointLess(r.sc, a, b)
if err != nil {
r.err = err
}
return less
}
func rangePointLess(sc *stmtctx.StatementContext, a, b point) (bool, error) {
cmp, err := a.value.CompareDatum(sc, &b.value)
if cmp != 0 {
return cmp < 0, nil
}
return rangePointEqualValueLess(a, b), errors.Trace(err)
}
func rangePointEqualValueLess(a, b point) bool {
if a.start && b.start {
return !a.excl && b.excl
} else if a.start {
return !a.excl && !b.excl
} else if b.start {
return a.excl || b.excl
}
return a.excl && !b.excl
}
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() []IntColumnRange {
return []IntColumnRange{{LowVal: math.MinInt64, HighVal: math.MaxInt64}}
}
// FullIntNewRange is used for table range. Since table range cannot accept MaxValueDatum as the max value.
// So we need to set it to MaxInt64.
func FullIntNewRange(isUnsigned bool) []*NewRange {
if isUnsigned {
return []*NewRange{{LowVal: []types.Datum{types.NewUintDatum(0)}, HighVal: []types.Datum{types.NewUintDatum(math.MaxUint64)}}}
}
return []*NewRange{{LowVal: []types.Datum{types.NewIntDatum(math.MinInt64)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}}
}
// FullNewRange is (-∞, +∞) for NewRange.
func FullNewRange() []*NewRange {
return []*NewRange{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}
}
// builder is the range builder struct.
type builder struct {
err error
sc *stmtctx.StatementContext
}
func (r *builder) build(expr expression.Expression) []point {
switch x := expr.(type) {
case *expression.Column:
return r.buildFromColumn(x)
case *expression.ScalarFunction:
return r.buildFromScalarFunc(x)
case *expression.Constant:
return r.buildFromConstant(x)
}
return fullRange
}
func (r *builder) buildFromConstant(expr *expression.Constant) []point {
dt, err := expr.Eval(nil)
if err != nil {
r.err = err
return nil
}
if dt.IsNull() {
return nil
}
val, err := dt.ToBool(r.sc)
if err != nil {
r.err = err
return nil
}
if val == 0 {
return nil
}
return fullRange
}
func (r *builder) buildFromColumn(expr *expression.Column) []point {
// column name expression is equivalent to column name is true.
startPoint1 := point{value: types.MinNotNullDatum(), start: true}
endPoint1 := point{excl: true}
endPoint1.value.SetInt64(0)
startPoint2 := point{excl: true, start: true}
startPoint2.value.SetInt64(0)
endPoint2 := point{value: types.MaxValueDatum()}
return []point{startPoint1, endPoint1, startPoint2, endPoint2}
}
func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []point {
// This has been checked that the binary operation is comparison operation, and one of
// the operand is column name expression.
var value types.Datum
var op string
var err error
if v, ok := expr.GetArgs()[0].(*expression.Constant); ok {
value, err = v.Eval(nil)
switch expr.FuncName.L {
case ast.GE:
op = ast.LE
case ast.GT:
op = ast.LT
case ast.LT:
op = ast.GT
case ast.LE:
op = ast.GE
default:
op = expr.FuncName.L
}
} else {
value, err = expr.GetArgs()[1].(*expression.Constant).Eval(nil)
op = expr.FuncName.L
}
if err != nil {
return nil
}
if value.IsNull() {
return nil
}
switch op {
case ast.EQ:
startPoint := point{value: value, start: true}
endPoint := point{value: value}
return []point{startPoint, endPoint}
case ast.NE:
startPoint1 := point{value: types.MinNotNullDatum(), start: true}
endPoint1 := point{value: value, excl: true}
startPoint2 := point{value: value, start: true, excl: true}
endPoint2 := point{value: types.MaxValueDatum()}
return []point{startPoint1, endPoint1, startPoint2, endPoint2}
case ast.LT:
startPoint := point{value: types.MinNotNullDatum(), start: true}
endPoint := point{value: value, excl: true}
return []point{startPoint, endPoint}
case ast.LE:
startPoint := point{value: types.MinNotNullDatum(), start: true}
endPoint := point{value: value}
return []point{startPoint, endPoint}
case ast.GT:
startPoint := point{value: value, start: true, excl: true}
endPoint := point{value: types.MaxValueDatum()}
return []point{startPoint, endPoint}
case ast.GE:
startPoint := point{value: value, start: true}
endPoint := point{value: types.MaxValueDatum()}
return []point{startPoint, endPoint}
}
return nil
}
func (r *builder) buildFromIsTrue(expr *expression.ScalarFunction, isNot int) []point {
if isNot == 1 {
// NOT TRUE range is {[null null] [0, 0]}
startPoint1 := point{start: true}
endPoint1 := point{}
startPoint2 := point{start: true}
startPoint2.value.SetInt64(0)
endPoint2 := point{}
endPoint2.value.SetInt64(0)
return []point{startPoint1, endPoint1, startPoint2, endPoint2}
}
// TRUE range is {[-inf 0) (0 +inf]}
startPoint1 := point{value: types.MinNotNullDatum(), start: true}
endPoint1 := point{excl: true}
endPoint1.value.SetInt64(0)
startPoint2 := point{excl: true, start: true}
startPoint2.value.SetInt64(0)
endPoint2 := point{value: types.MaxValueDatum()}
return []point{startPoint1, endPoint1, startPoint2, endPoint2}
}
func (r *builder) buildFromIsFalse(expr *expression.ScalarFunction, isNot int) []point {
if isNot == 1 {
// NOT FALSE range is {[-inf, 0), (0, +inf], [null, null]}
startPoint1 := point{start: true}
endPoint1 := point{excl: true}
endPoint1.value.SetInt64(0)
startPoint2 := point{start: true, excl: true}
startPoint2.value.SetInt64(0)
endPoint2 := point{value: types.MaxValueDatum()}
return []point{startPoint1, endPoint1, startPoint2, endPoint2}
}
// FALSE range is {[0, 0]}
startPoint := point{start: true}
startPoint.value.SetInt64(0)
endPoint := point{}
endPoint.value.SetInt64(0)
return []point{startPoint, endPoint}
}
func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]point, bool) {
list := expr.GetArgs()[1:]
rangePoints := make([]point, 0, len(list)*2)
hasNull := false
for _, e := range list {
v, ok := e.(*expression.Constant)
if !ok {
r.err = ErrUnsupportedType.Gen("expr:%v is not constant", e)
return fullRange, hasNull
}
dt, err := v.Eval(nil)
if err != nil {
r.err = ErrUnsupportedType.Gen("expr:%v is not evaluated", e)
return fullRange, hasNull
}
if dt.IsNull() {
hasNull = true
continue
}
startPoint := point{value: types.NewDatum(dt.GetValue()), start: true}
endPoint := point{value: types.NewDatum(dt.GetValue())}
rangePoints = append(rangePoints, startPoint, endPoint)
}
sorter := pointSorter{points: rangePoints, sc: r.sc}
sort.Sort(&sorter)
if sorter.err != nil {
r.err = sorter.err
}
// check and remove duplicates
curPos, frontPos := 0, 0
for frontPos < len(rangePoints) {
if rangePoints[curPos].start == rangePoints[frontPos].start {
frontPos++
} else {
curPos++
rangePoints[curPos] = rangePoints[frontPos]
frontPos++
}
}
if curPos > 0 {
curPos++
}
return rangePoints[:curPos], hasNull
}
func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []point {
pdt, err := expr.GetArgs()[1].(*expression.Constant).Eval(nil)
if err != nil {
r.err = errors.Trace(err)
return fullRange
}
pattern, err := pdt.ToString()
if err != nil {
r.err = errors.Trace(err)
return fullRange
}
if pattern == "" {
startPoint := point{value: types.NewStringDatum(""), start: true}
endPoint := point{value: types.NewStringDatum("")}
return []point{startPoint, endPoint}
}
lowValue := make([]byte, 0, len(pattern))
edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(nil)
if err != nil {
r.err = errors.Trace(err)
return fullRange
}
escape := byte(edt.GetInt64())
var exclude bool
isExactMatch := true
for i := 0; i < len(pattern); i++ {
if pattern[i] == escape {
i++
if i < len(pattern) {
lowValue = append(lowValue, pattern[i])
} else {
lowValue = append(lowValue, escape)
}
continue
}
if pattern[i] == '%' {
// Get the prefix.
isExactMatch = false
break
} else if pattern[i] == '_' {
// Get the prefix, but exclude the prefix.
// e.g., "abc_x", the start point exclude "abc",
// because the string length is more than 3.
exclude = true
isExactMatch = false
break
}
lowValue = append(lowValue, pattern[i])
}
if len(lowValue) == 0 {
return []point{{value: types.MinNotNullDatum(), start: true}, {value: types.MaxValueDatum()}}
}
if isExactMatch {
val := types.NewStringDatum(string(lowValue))
return []point{{value: val, start: true}, {value: val}}
}
startPoint := point{start: true, excl: exclude}
startPoint.value.SetBytesAsString(lowValue)
highValue := make([]byte, len(lowValue))
copy(highValue, lowValue)
endPoint := point{excl: true}
for i := len(highValue) - 1; i >= 0; i-- {
// Make the end point value more than the start point value,
// and the length of the end point value is the same as the length of the start point value.
// e.g., the start point value is "abc", so the end point value is "abd".
highValue[i]++
if highValue[i] != 0 {
endPoint.value.SetBytesAsString(highValue)
break
}
// If highValue[i] is 255 and highValue[i]++ is 0, then the end point value is max value.
if i == 0 {
endPoint.value = types.MaxValueDatum()
}
}
return []point{startPoint, endPoint}
}
func (r *builder) buildFromNot(expr *expression.ScalarFunction) []point {
switch n := expr.FuncName.L; n {
case ast.IsTruth:
return r.buildFromIsTrue(expr, 1)
case ast.IsFalsity:
return r.buildFromIsFalse(expr, 1)
case ast.In:
var (
isUnsignedIntCol bool
nonNegativePos int
)
rangePoints, hasNull := r.buildFromIn(expr)
if hasNull {
return nil
}
if x, ok := expr.GetArgs()[0].(*expression.Column); ok {
isUnsignedIntCol = mysql.HasUnsignedFlag(x.RetType.Flag) && mysql.IsIntegerType(x.RetType.Tp)
}
// negative ranges can be directly ignored for unsigned int columns.
if isUnsignedIntCol {
for nonNegativePos = 0; nonNegativePos < len(rangePoints); nonNegativePos += 2 {
if rangePoints[nonNegativePos].value.Kind() == types.KindUint64 || rangePoints[nonNegativePos].value.GetInt64() >= 0 {
break
}
}
rangePoints = rangePoints[nonNegativePos:]
}
retRangePoints := make([]point, 0, 2+len(rangePoints))
previousValue := types.Datum{}
for i := 0; i < len(rangePoints); i += 2 {
retRangePoints = append(retRangePoints, point{value: previousValue, start: true, excl: true})
retRangePoints = append(retRangePoints, point{value: rangePoints[i].value, excl: true})
previousValue = rangePoints[i].value
}
// Append the interval (last element, max value].
retRangePoints = append(retRangePoints, point{value: previousValue, start: true, excl: true})
retRangePoints = append(retRangePoints, point{value: types.MaxValueDatum()})
return retRangePoints
case ast.Like:
// Pattern not like is not supported.
r.err = ErrUnsupportedType.Gen("NOT LIKE is not supported.")
return fullRange
case ast.IsNull:
startPoint := point{value: types.MinNotNullDatum(), start: true}
endPoint := point{value: types.MaxValueDatum()}
return []point{startPoint, endPoint}
}
return nil
}
func (r *builder) buildFromScalarFunc(expr *expression.ScalarFunction) []point {
switch op := expr.FuncName.L; op {
case ast.GE, ast.GT, ast.LT, ast.LE, ast.EQ, ast.NE:
return r.buildFormBinOp(expr)
case ast.LogicAnd:
return r.intersection(r.build(expr.GetArgs()[0]), r.build(expr.GetArgs()[1]))
case ast.LogicOr:
return r.union(r.build(expr.GetArgs()[0]), r.build(expr.GetArgs()[1]))
case ast.IsTruth:
return r.buildFromIsTrue(expr, 0)
case ast.IsFalsity:
return r.buildFromIsFalse(expr, 0)
case ast.In:
retPoints, _ := r.buildFromIn(expr)
return retPoints
case ast.Like:
return r.newBuildFromPatternLike(expr)
case ast.IsNull:
startPoint := point{start: true}
endPoint := point{}
return []point{startPoint, endPoint}
case ast.UnaryNot:
return r.buildFromNot(expr.GetArgs()[0].(*expression.ScalarFunction))
}
return nil
}
func (r *builder) intersection(a, b []point) []point {
return r.merge(a, b, false)
}
func (r *builder) union(a, b []point) []point {
return r.merge(a, b, true)
}
func (r *builder) merge(a, b []point, union bool) []point {
sorter := pointSorter{points: append(a, b...), sc: r.sc}
sort.Sort(&sorter)
if sorter.err != nil {
r.err = sorter.err
return nil
}
var (
inRangeCount int
requiredInRangeCount int
)
if union {
requiredInRangeCount = 1
} else {
requiredInRangeCount = 2
}
merged := make([]point, 0, len(sorter.points))
for _, val := range sorter.points {
if val.start {
inRangeCount++
if inRangeCount == requiredInRangeCount {
// just reached the required in range count, a new range started.
merged = append(merged, val)
}
} else {
if inRangeCount == requiredInRangeCount {
// just about to leave the required in range count, the range is ended.
merged = append(merged, val)
}
inRangeCount--
}
}
return merged
}
Loading...
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
1
https://gitee.com/zhoujin826/tidb.git
git@gitee.com:zhoujin826/tidb.git
zhoujin826
tidb
tidb
v2.0.5

搜索帮助