expression: Implement VecEvalBool to prepare for vectorized selection operator (#12175)
This commit is contained in:
@ -257,6 +257,15 @@ func (rig *rangeInt64Gener) gen() interface{} {
|
||||
return int64(rand.Intn(rig.end-rig.begin) + rig.begin)
|
||||
}
|
||||
|
||||
// numStrGener is used to generate number strings.
|
||||
type numStrGener struct {
|
||||
rangeInt64Gener
|
||||
}
|
||||
|
||||
func (g *numStrGener) gen() interface{} {
|
||||
return fmt.Sprintf("%v", g.rangeInt64Gener.gen())
|
||||
}
|
||||
|
||||
// randLenStrGener is used to generate strings whose lengths are in [lenBegin, lenEnd).
|
||||
type randLenStrGener struct {
|
||||
lenBegin int
|
||||
@ -762,3 +771,114 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func genVecEvalBool(numCols int, colTypes []types.EvalType) (CNFExprs, *chunk.Chunk) {
|
||||
eTypes := []types.EvalType{types.ETInt, types.ETReal, types.ETDecimal, types.ETString, types.ETTimestamp, types.ETDatetime, types.ETDuration}
|
||||
gens := make([]dataGenerator, 0, len(eTypes))
|
||||
for _, eType := range eTypes {
|
||||
if eType == types.ETString {
|
||||
gens = append(gens, &numStrGener{rangeInt64Gener{0, 10}})
|
||||
} else {
|
||||
gens = append(gens, &defaultGener{nullRation: 0.05, eType: eType})
|
||||
}
|
||||
}
|
||||
|
||||
ts := make([]types.EvalType, 0, numCols)
|
||||
gs := make([]dataGenerator, 0, numCols)
|
||||
fts := make([]*types.FieldType, 0, numCols)
|
||||
for i := 0; i < numCols; i++ {
|
||||
idx := rand.Intn(len(eTypes))
|
||||
if colTypes != nil {
|
||||
for j := range eTypes {
|
||||
if colTypes[i] == eTypes[j] {
|
||||
idx = j
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
ts = append(ts, eTypes[idx])
|
||||
gs = append(gs, gens[idx])
|
||||
fts = append(fts, eType2FieldType(eTypes[idx]))
|
||||
}
|
||||
|
||||
input := chunk.New(fts, 1024, 1024)
|
||||
exprs := make(CNFExprs, 0, numCols)
|
||||
for i := 0; i < numCols; i++ {
|
||||
fillColumn(ts[i], input, i, vecExprBenchCase{geners: gs})
|
||||
exprs = append(exprs, &Column{Index: i, RetType: fts[i]})
|
||||
}
|
||||
return exprs, input
|
||||
}
|
||||
|
||||
func (s *testEvaluatorSuite) TestVecEvalBool(c *C) {
|
||||
ctx := mock.NewContext()
|
||||
for numCols := 1; numCols <= 10; numCols++ {
|
||||
for round := 0; round < 64; round++ {
|
||||
exprs, input := genVecEvalBool(numCols, nil)
|
||||
selected, nulls, err := VecEvalBool(ctx, exprs, input, nil, nil)
|
||||
c.Assert(err, IsNil)
|
||||
it := chunk.NewIterator4Chunk(input)
|
||||
i := 0
|
||||
for row := it.Begin(); row != it.End(); row = it.Next() {
|
||||
ok, null, err := EvalBool(mock.NewContext(), exprs, row)
|
||||
c.Assert(err, IsNil)
|
||||
c.Assert(null, Equals, nulls[i])
|
||||
c.Assert(ok, Equals, selected[i])
|
||||
i++
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkVecEvalBool(b *testing.B) {
|
||||
ctx := mock.NewContext()
|
||||
selected := make([]bool, 0, 1024)
|
||||
nulls := make([]bool, 0, 1024)
|
||||
eTypes := []types.EvalType{types.ETInt, types.ETReal, types.ETDecimal, types.ETString, types.ETTimestamp, types.ETDatetime, types.ETDuration}
|
||||
tNames := []string{"int", "real", "decimal", "string", "timestamp", "datetime", "duration"}
|
||||
for numCols := 1; numCols <= 3; numCols++ {
|
||||
typeCombination := make([]types.EvalType, numCols)
|
||||
var combFunc func(nCols int)
|
||||
combFunc = func(nCols int) {
|
||||
if nCols == 0 {
|
||||
name := ""
|
||||
for _, t := range typeCombination {
|
||||
for i := range eTypes {
|
||||
if t == eTypes[i] {
|
||||
name += tNames[t] + "/"
|
||||
}
|
||||
}
|
||||
}
|
||||
exprs, input := genVecEvalBool(numCols, typeCombination)
|
||||
b.Run("Vec-"+name, func(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _, err := VecEvalBool(ctx, exprs, input, selected, nulls)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
})
|
||||
b.Run("Row-"+name, func(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
it := chunk.NewIterator4Chunk(input)
|
||||
for row := it.Begin(); row != it.End(); row = it.Next() {
|
||||
_, _, err := EvalBool(ctx, exprs, row)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
return
|
||||
}
|
||||
for _, eType := range eTypes {
|
||||
typeCombination[nCols-1] = eType
|
||||
combFunc(nCols - 1)
|
||||
}
|
||||
}
|
||||
|
||||
combFunc(numCols)
|
||||
}
|
||||
}
|
||||
|
||||
@ -45,6 +45,8 @@ func newLocalSliceBuffer(initCap int) *localSliceBuffer {
|
||||
return &localSliceBuffer{buffers: make([]*chunk.Column, initCap)}
|
||||
}
|
||||
|
||||
var globalColumnAllocator = newLocalSliceBuffer(1024)
|
||||
|
||||
func newBuffer(evalType types.EvalType, capacity int) (*chunk.Column, error) {
|
||||
switch evalType {
|
||||
case types.ETInt:
|
||||
|
||||
@ -16,6 +16,7 @@ package expression
|
||||
import (
|
||||
goJSON "encoding/json"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/pingcap/parser/ast"
|
||||
"github.com/pingcap/parser/model"
|
||||
@ -208,6 +209,126 @@ func EvalBool(ctx sessionctx.Context, exprList CNFExprs, row chunk.Row) (bool, b
|
||||
return true, false, nil
|
||||
}
|
||||
|
||||
var (
|
||||
selPool = sync.Pool{
|
||||
New: func() interface{} {
|
||||
return make([]int, 1024)
|
||||
},
|
||||
}
|
||||
)
|
||||
|
||||
// VecEvalBool does the same thing as EvalBool but it works in a vectorized manner.
|
||||
func VecEvalBool(ctx sessionctx.Context, exprList CNFExprs, input *chunk.Chunk, selected, nulls []bool) ([]bool, []bool, error) {
|
||||
n := input.NumRows()
|
||||
selected = selected[:0]
|
||||
nulls = nulls[:0]
|
||||
for i := 0; i < n; i++ {
|
||||
selected = append(selected, false)
|
||||
nulls = append(nulls, false)
|
||||
}
|
||||
|
||||
sel := selPool.Get().([]int)
|
||||
defer selPool.Put(sel)
|
||||
sel = sel[:0]
|
||||
for i := 0; i < n; i++ {
|
||||
sel = append(sel, i)
|
||||
}
|
||||
defer input.SetSel(input.Sel())
|
||||
input.SetSel(sel)
|
||||
|
||||
for _, expr := range exprList {
|
||||
eType := expr.GetType().EvalType()
|
||||
buf, err := globalColumnAllocator.get(eType, n)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
if err := vecEval(ctx, expr, input, buf); err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
isEQCondFromIn := IsEQCondFromIn(expr)
|
||||
hasUnsignedFlag := mysql.HasUnsignedFlag(expr.GetType().Flag)
|
||||
d, j := types.Datum{}, 0
|
||||
for i := range sel {
|
||||
if buf.IsNull(i) {
|
||||
if !isEQCondFromIn {
|
||||
nulls[sel[i]] = false
|
||||
continue
|
||||
}
|
||||
// In this case, we set this row to null and let it pass this filter.
|
||||
// The null flag may be set to false later by other expressions in some cases.
|
||||
nulls[sel[i]] = true
|
||||
sel[j] = sel[i]
|
||||
j++
|
||||
continue
|
||||
}
|
||||
|
||||
switch eType {
|
||||
case types.ETInt:
|
||||
if hasUnsignedFlag {
|
||||
d.SetUint64(buf.GetUint64(i))
|
||||
} else {
|
||||
d.SetInt64(buf.GetInt64(i))
|
||||
}
|
||||
case types.ETReal:
|
||||
d.SetFloat64(buf.GetFloat64(i))
|
||||
case types.ETDuration:
|
||||
d.SetMysqlDuration(buf.GetDuration(i, 0))
|
||||
case types.ETDatetime, types.ETTimestamp:
|
||||
d.SetMysqlTime(buf.GetTime(i))
|
||||
case types.ETString:
|
||||
d.SetString(buf.GetString(i))
|
||||
case types.ETJson:
|
||||
d.SetMysqlJSON(buf.GetJSON(i))
|
||||
case types.ETDecimal:
|
||||
d.SetMysqlDecimal(buf.GetDecimal(i))
|
||||
}
|
||||
|
||||
b, err := d.ToBool(ctx.GetSessionVars().StmtCtx)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
if b == 0 {
|
||||
continue
|
||||
}
|
||||
sel[j] = sel[i] // this row passes this filter
|
||||
j++
|
||||
}
|
||||
sel = sel[:j]
|
||||
input.SetSel(sel)
|
||||
globalColumnAllocator.put(buf)
|
||||
}
|
||||
|
||||
for _, i := range sel {
|
||||
if !nulls[i] {
|
||||
selected[i] = true
|
||||
}
|
||||
}
|
||||
|
||||
return selected, nulls, nil
|
||||
}
|
||||
|
||||
func vecEval(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, result *chunk.Column) (err error) {
|
||||
switch expr.GetType().EvalType() {
|
||||
case types.ETInt:
|
||||
err = expr.VecEvalInt(ctx, input, result)
|
||||
case types.ETReal:
|
||||
err = expr.VecEvalReal(ctx, input, result)
|
||||
case types.ETDuration:
|
||||
err = expr.VecEvalDuration(ctx, input, result)
|
||||
case types.ETDatetime, types.ETTimestamp:
|
||||
err = expr.VecEvalTime(ctx, input, result)
|
||||
case types.ETString:
|
||||
err = expr.VecEvalString(ctx, input, result)
|
||||
case types.ETJson:
|
||||
err = expr.VecEvalJSON(ctx, input, result)
|
||||
case types.ETDecimal:
|
||||
err = expr.VecEvalDecimal(ctx, input, result)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// composeConditionWithBinaryOp composes condition with binary operator into a balance deep tree, which benefits a lot for pb decoder/encoder.
|
||||
func composeConditionWithBinaryOp(ctx sessionctx.Context, conditions []Expression, funcName string) Expression {
|
||||
length := len(conditions)
|
||||
|
||||
@ -612,12 +612,17 @@ func (c *Column) CopyReconstruct(sel []int, dst *Column) *Column {
|
||||
|
||||
if c.isFixed() {
|
||||
elemLen := len(c.elemBuf)
|
||||
dst.elemBuf = make([]byte, elemLen)
|
||||
for _, i := range sel {
|
||||
dst.appendNullBitmap(!c.IsNull(i))
|
||||
dst.data = append(dst.data, c.data[i*elemLen:i*elemLen+elemLen]...)
|
||||
dst.length++
|
||||
}
|
||||
} else {
|
||||
dst.elemBuf = nil
|
||||
if len(dst.offsets) == 0 {
|
||||
dst.offsets = append(dst.offsets, 0)
|
||||
}
|
||||
for _, i := range sel {
|
||||
dst.appendNullBitmap(!c.IsNull(i))
|
||||
start, end := c.offsets[i], c.offsets[i+1]
|
||||
|
||||
Reference in New Issue
Block a user