Skip to content

Commit

Permalink
cherry pick pingcap#19621 to release-3.0
Browse files Browse the repository at this point in the history
Signed-off-by: ti-srebot <[email protected]>
  • Loading branch information
wshwsh12 authored and ti-srebot committed Sep 9, 2020
1 parent 40a147c commit 14f2372
Show file tree
Hide file tree
Showing 14 changed files with 997 additions and 71 deletions.
2 changes: 1 addition & 1 deletion executor/reload_expr_pushdown_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ var funcName2Alias = map[string]string{
"case": ast.Case,
"regexp": ast.Regexp,
"is null": ast.IsNull,
"is true": ast.IsTruth,
"is true": ast.IsTruthWithoutNull,
"is false": ast.IsFalsity,
"values": ast.Values,
"bit_count": ast.BitCount,
Expand Down
112 changes: 77 additions & 35 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -519,41 +519,42 @@ var funcs = map[string]functionClass{
ast.GetLock: &lockFunctionClass{baseFunctionClass{ast.GetLock, 2, 2}},
ast.ReleaseLock: &releaseLockFunctionClass{baseFunctionClass{ast.ReleaseLock, 1, 1}},

ast.LogicAnd: &logicAndFunctionClass{baseFunctionClass{ast.LogicAnd, 2, 2}},
ast.LogicOr: &logicOrFunctionClass{baseFunctionClass{ast.LogicOr, 2, 2}},
ast.LogicXor: &logicXorFunctionClass{baseFunctionClass{ast.LogicXor, 2, 2}},
ast.GE: &compareFunctionClass{baseFunctionClass{ast.GE, 2, 2}, opcode.GE},
ast.LE: &compareFunctionClass{baseFunctionClass{ast.LE, 2, 2}, opcode.LE},
ast.EQ: &compareFunctionClass{baseFunctionClass{ast.EQ, 2, 2}, opcode.EQ},
ast.NE: &compareFunctionClass{baseFunctionClass{ast.NE, 2, 2}, opcode.NE},
ast.LT: &compareFunctionClass{baseFunctionClass{ast.LT, 2, 2}, opcode.LT},
ast.GT: &compareFunctionClass{baseFunctionClass{ast.GT, 2, 2}, opcode.GT},
ast.NullEQ: &compareFunctionClass{baseFunctionClass{ast.NullEQ, 2, 2}, opcode.NullEQ},
ast.Plus: &arithmeticPlusFunctionClass{baseFunctionClass{ast.Plus, 2, 2}},
ast.Minus: &arithmeticMinusFunctionClass{baseFunctionClass{ast.Minus, 2, 2}},
ast.Mod: &arithmeticModFunctionClass{baseFunctionClass{ast.Mod, 2, 2}},
ast.Div: &arithmeticDivideFunctionClass{baseFunctionClass{ast.Div, 2, 2}},
ast.Mul: &arithmeticMultiplyFunctionClass{baseFunctionClass{ast.Mul, 2, 2}},
ast.IntDiv: &arithmeticIntDivideFunctionClass{baseFunctionClass{ast.IntDiv, 2, 2}},
ast.BitNeg: &bitNegFunctionClass{baseFunctionClass{ast.BitNeg, 1, 1}},
ast.And: &bitAndFunctionClass{baseFunctionClass{ast.And, 2, 2}},
ast.LeftShift: &leftShiftFunctionClass{baseFunctionClass{ast.LeftShift, 2, 2}},
ast.RightShift: &rightShiftFunctionClass{baseFunctionClass{ast.RightShift, 2, 2}},
ast.UnaryNot: &unaryNotFunctionClass{baseFunctionClass{ast.UnaryNot, 1, 1}},
ast.Or: &bitOrFunctionClass{baseFunctionClass{ast.Or, 2, 2}},
ast.Xor: &bitXorFunctionClass{baseFunctionClass{ast.Xor, 2, 2}},
ast.UnaryMinus: &unaryMinusFunctionClass{baseFunctionClass{ast.UnaryMinus, 1, 1}},
ast.In: &inFunctionClass{baseFunctionClass{ast.In, 2, -1}},
ast.IsTruth: &isTrueOrFalseFunctionClass{baseFunctionClass{ast.IsTruth, 1, 1}, opcode.IsTruth, false},
ast.IsFalsity: &isTrueOrFalseFunctionClass{baseFunctionClass{ast.IsFalsity, 1, 1}, opcode.IsFalsity, false},
ast.Like: &likeFunctionClass{baseFunctionClass{ast.Like, 3, 3}},
ast.Regexp: &regexpFunctionClass{baseFunctionClass{ast.Regexp, 2, 2}},
ast.Case: &caseWhenFunctionClass{baseFunctionClass{ast.Case, 1, -1}},
ast.RowFunc: &rowFunctionClass{baseFunctionClass{ast.RowFunc, 2, -1}},
ast.SetVar: &setVarFunctionClass{baseFunctionClass{ast.SetVar, 2, 2}},
ast.GetVar: &getVarFunctionClass{baseFunctionClass{ast.GetVar, 1, 1}},
ast.BitCount: &bitCountFunctionClass{baseFunctionClass{ast.BitCount, 1, 1}},
ast.GetParam: &getParamFunctionClass{baseFunctionClass{ast.GetParam, 1, 1}},
ast.LogicAnd: &logicAndFunctionClass{baseFunctionClass{ast.LogicAnd, 2, 2}},
ast.LogicOr: &logicOrFunctionClass{baseFunctionClass{ast.LogicOr, 2, 2}},
ast.LogicXor: &logicXorFunctionClass{baseFunctionClass{ast.LogicXor, 2, 2}},
ast.GE: &compareFunctionClass{baseFunctionClass{ast.GE, 2, 2}, opcode.GE},
ast.LE: &compareFunctionClass{baseFunctionClass{ast.LE, 2, 2}, opcode.LE},
ast.EQ: &compareFunctionClass{baseFunctionClass{ast.EQ, 2, 2}, opcode.EQ},
ast.NE: &compareFunctionClass{baseFunctionClass{ast.NE, 2, 2}, opcode.NE},
ast.LT: &compareFunctionClass{baseFunctionClass{ast.LT, 2, 2}, opcode.LT},
ast.GT: &compareFunctionClass{baseFunctionClass{ast.GT, 2, 2}, opcode.GT},
ast.NullEQ: &compareFunctionClass{baseFunctionClass{ast.NullEQ, 2, 2}, opcode.NullEQ},
ast.Plus: &arithmeticPlusFunctionClass{baseFunctionClass{ast.Plus, 2, 2}},
ast.Minus: &arithmeticMinusFunctionClass{baseFunctionClass{ast.Minus, 2, 2}},
ast.Mod: &arithmeticModFunctionClass{baseFunctionClass{ast.Mod, 2, 2}},
ast.Div: &arithmeticDivideFunctionClass{baseFunctionClass{ast.Div, 2, 2}},
ast.Mul: &arithmeticMultiplyFunctionClass{baseFunctionClass{ast.Mul, 2, 2}},
ast.IntDiv: &arithmeticIntDivideFunctionClass{baseFunctionClass{ast.IntDiv, 2, 2}},
ast.BitNeg: &bitNegFunctionClass{baseFunctionClass{ast.BitNeg, 1, 1}},
ast.And: &bitAndFunctionClass{baseFunctionClass{ast.And, 2, 2}},
ast.LeftShift: &leftShiftFunctionClass{baseFunctionClass{ast.LeftShift, 2, 2}},
ast.RightShift: &rightShiftFunctionClass{baseFunctionClass{ast.RightShift, 2, 2}},
ast.UnaryNot: &unaryNotFunctionClass{baseFunctionClass{ast.UnaryNot, 1, 1}},
ast.Or: &bitOrFunctionClass{baseFunctionClass{ast.Or, 2, 2}},
ast.Xor: &bitXorFunctionClass{baseFunctionClass{ast.Xor, 2, 2}},
ast.UnaryMinus: &unaryMinusFunctionClass{baseFunctionClass{ast.UnaryMinus, 1, 1}},
ast.In: &inFunctionClass{baseFunctionClass{ast.In, 2, -1}},
ast.IsTruthWithoutNull: &isTrueOrFalseFunctionClass{baseFunctionClass{ast.IsTruthWithoutNull, 1, 1}, opcode.IsTruth, false},
ast.IsTruthWithNull: &isTrueOrFalseFunctionClass{baseFunctionClass{ast.IsTruthWithNull, 1, 1}, opcode.IsTruth, true},
ast.IsFalsity: &isTrueOrFalseFunctionClass{baseFunctionClass{ast.IsFalsity, 1, 1}, opcode.IsFalsity, false},
ast.Like: &likeFunctionClass{baseFunctionClass{ast.Like, 3, 3}},
ast.Regexp: &regexpFunctionClass{baseFunctionClass{ast.Regexp, 2, 2}},
ast.Case: &caseWhenFunctionClass{baseFunctionClass{ast.Case, 1, -1}},
ast.RowFunc: &rowFunctionClass{baseFunctionClass{ast.RowFunc, 2, -1}},
ast.SetVar: &setVarFunctionClass{baseFunctionClass{ast.SetVar, 2, 2}},
ast.GetVar: &getVarFunctionClass{baseFunctionClass{ast.GetVar, 1, 1}},
ast.BitCount: &bitCountFunctionClass{baseFunctionClass{ast.BitCount, 1, 1}},
ast.GetParam: &getParamFunctionClass{baseFunctionClass{ast.GetParam, 1, 1}},

// encryption and compression functions
ast.AesDecrypt: &aesDecryptFunctionClass{baseFunctionClass{ast.AesDecrypt, 2, 3}},
Expand Down Expand Up @@ -607,4 +608,45 @@ var funcs = map[string]functionClass{
ast.TiDBIsDDLOwner: &tidbIsDDLOwnerFunctionClass{baseFunctionClass{ast.TiDBIsDDLOwner, 0, 0}},
ast.TiDBParseTso: &tidbParseTsoFunctionClass{baseFunctionClass{ast.TiDBParseTso, 1, 1}},
ast.TiDBDecodePlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodePlan, 1, 1}},
<<<<<<< HEAD
=======

// TiDB Sequence function.
ast.NextVal: &nextValFunctionClass{baseFunctionClass{ast.NextVal, 1, 1}},
ast.LastVal: &lastValFunctionClass{baseFunctionClass{ast.LastVal, 1, 1}},
ast.SetVal: &setValFunctionClass{baseFunctionClass{ast.SetVal, 2, 2}},
}

// IsFunctionSupported check if given function name is a builtin sql function.
func IsFunctionSupported(name string) bool {
_, ok := funcs[name]
return ok
}

// GetBuiltinList returns a list of builtin functions
func GetBuiltinList() []string {
res := make([]string, 0, len(funcs))
notImplementedFunctions := []string{ast.RowFunc, ast.IsTruthWithNull}
for funcName := range funcs {
skipFunc := false
// Skip not implemented functions
for _, notImplFunc := range notImplementedFunctions {
if funcName == notImplFunc {
skipFunc = true
}
}
// Skip literal functions
// (their names are not readable: 'tidb`.(dateliteral, for example)
// See: https://github.com/pingcap/parser/pull/591
if strings.HasPrefix(funcName, "'tidb`.(") {
skipFunc = true
}
if skipFunc {
continue
}
res = append(res, funcName)
}
sort.Strings(res)
return res
>>>>>>> 0c36203... expression: add new scalar function IsTruthWithNull (#19621)
}
2 changes: 1 addition & 1 deletion expression/builtin_op_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -565,7 +565,7 @@ func (s *testEvaluatorSuite) TestIsTrueOrFalse(c *C) {
}

for _, tc := range testCases {
isTrueSig, err := funcs[ast.IsTruth].getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(tc.args...)))
isTrueSig, err := funcs[ast.IsTruthWithoutNull].getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(tc.args...)))
c.Assert(err, IsNil)
c.Assert(isTrueSig, NotNil)

Expand Down
194 changes: 194 additions & 0 deletions expression/builtin_op_vec_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,194 @@
// 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 expression

import (
"math"
"testing"

. "github.com/pingcap/check"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/mock"
)

var vecBuiltinOpCases = map[string][]vecExprBenchCase{
ast.IsTruthWithoutNull: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
},
ast.IsFalsity: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
},
ast.LogicOr: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal, types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETDuration}},
},
ast.LogicXor: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
},
ast.Xor: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
},
ast.LogicAnd: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal, types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETDuration}},
},
ast.Or: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
},
ast.BitNeg: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
},
ast.UnaryNot: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
},
ast.And: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: makeBinaryLogicOpDataGeners()},
},
ast.RightShift: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}},
},
ast.LeftShift: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}},
},
ast.UnaryMinus: {
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETDecimal, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
{
retEvalType: types.ETInt,
childrenTypes: []types.EvalType{types.ETInt},
childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag}},
geners: []dataGenerator{newRangeInt64Gener(0, math.MaxInt64)},
},
},
ast.IsNull: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDuration}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDatetime}},
},
}

// givenValsGener returns the items sequentially from the slice given at
// the construction time. If this slice is exhausted, it falls back to
// the fallback generator.
type givenValsGener struct {
given []interface{}
idx int
fallback dataGenerator
}

func (g *givenValsGener) gen() interface{} {
if g.idx >= len(g.given) {
return g.fallback.gen()
}
v := g.given[g.idx]
g.idx++
return v
}

func makeGivenValsOrDefaultGener(vals []interface{}, eType types.EvalType) *givenValsGener {
g := &givenValsGener{}
g.given = vals
g.fallback = newDefaultGener(0.2, eType)
return g
}

func makeBinaryLogicOpDataGeners() []dataGenerator {
// TODO: rename this to makeBinaryOpDataGenerator, since the BIT ops are also using it?
pairs := [][]interface{}{
{nil, nil},
{0, nil},
{nil, 0},
{1, nil},
{nil, 1},
{0, 0},
{0, 1},
{1, 0},
{1, 1},
{-1, 1},
}

maybeToInt64 := func(v interface{}) interface{} {
if v == nil {
return nil
}
return int64(v.(int))
}

n := len(pairs)
arg0s := make([]interface{}, n)
arg1s := make([]interface{}, n)
for i, p := range pairs {
arg0s[i] = maybeToInt64(p[0])
arg1s[i] = maybeToInt64(p[1])
}
return []dataGenerator{
makeGivenValsOrDefaultGener(arg0s, types.ETInt),
makeGivenValsOrDefaultGener(arg1s, types.ETInt)}
}

func (s *testEvaluatorSuite) TestVectorizedBuiltinOpFunc(c *C) {
testVectorizedBuiltinFunc(c, vecBuiltinOpCases)
}

func BenchmarkVectorizedBuiltinOpFunc(b *testing.B) {
benchmarkVectorizedBuiltinFunc(b, vecBuiltinOpCases)
}

func (s *testEvaluatorSuite) TestBuiltinUnaryMinusIntSig(c *C) {
ctx := mock.NewContext()
ft := eType2FieldType(types.ETInt)
col0 := &Column{RetType: ft, Index: 0}
f, err := funcs[ast.UnaryMinus].getFunction(ctx, []Expression{col0})
c.Assert(err, IsNil)
input := chunk.NewChunkWithCapacity([]*types.FieldType{ft}, 1024)
result := chunk.NewColumn(ft, 1024)

c.Assert(mysql.HasUnsignedFlag(col0.GetType().Flag), IsFalse)
input.AppendInt64(0, 233333)
c.Assert(f.vecEvalInt(input, result), IsNil)
c.Assert(result.GetInt64(0), Equals, int64(-233333))
input.Reset()
input.AppendInt64(0, math.MinInt64)
c.Assert(f.vecEvalInt(input, result), NotNil)
input.Column(0).SetNull(0, true)
c.Assert(f.vecEvalInt(input, result), IsNil)
c.Assert(result.IsNull(0), IsTrue)

col0.GetType().Flag |= mysql.UnsignedFlag
c.Assert(mysql.HasUnsignedFlag(col0.GetType().Flag), IsTrue)
input.Reset()
input.AppendUint64(0, 233333)
c.Assert(f.vecEvalInt(input, result), IsNil)
c.Assert(result.GetInt64(0), Equals, int64(-233333))
input.Reset()
input.AppendUint64(0, -(math.MinInt64)+1)
c.Assert(f.vecEvalInt(input, result), NotNil)
input.Column(0).SetNull(0, true)
c.Assert(f.vecEvalInt(input, result), IsNil)
c.Assert(result.IsNull(0), IsTrue)
}
Loading

0 comments on commit 14f2372

Please sign in to comment.