expression: Implement VecEvalBool to prepare for vectorized selection operator (#12175)

This commit is contained in:
Yuanjia Zhang
2019-09-16 17:47:52 +08:00
committed by GitHub
parent 04292a00de
commit 4e545cfa58
4 changed files with 248 additions and 0 deletions

View File

@ -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)
}
}

View File

@ -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:

View File

@ -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)

View File

@ -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]