Skip to content

Commit

Permalink
expression: refactor builtinRegexpSig to remove duplicate code (pingc…
Browse files Browse the repository at this point in the history
  • Loading branch information
k-ye authored and qw4990 committed Oct 22, 2019
1 parent 05d0ad0 commit 2d28a27
Show file tree
Hide file tree
Showing 4 changed files with 162 additions and 62 deletions.
18 changes: 16 additions & 2 deletions expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -444,6 +444,15 @@ func (g *dataStrGener) gen() interface{} {
return fmt.Sprintf("%d:%d:%d", hour, minute, second)
}

// constStrGener always returns the given string
type constStrGener struct {
s string
}

func (g *constStrGener) gen() interface{} {
return g.s
}

type randDurInt struct{}

func (g *randDurInt) gen() interface{} {
Expand Down Expand Up @@ -472,11 +481,16 @@ type vecExprBenchCase struct {
type vecExprBenchCases map[string][]vecExprBenchCase

func fillColumn(eType types.EvalType, chk *chunk.Chunk, colIdx int, testCase vecExprBenchCase) {
batchSize := 1024
var gen dataGenerator
if len(testCase.geners) > colIdx && testCase.geners[colIdx] != nil {
gen = testCase.geners[colIdx]
} else {
}
fillColumnWithGener(eType, chk, colIdx, gen)
}

func fillColumnWithGener(eType types.EvalType, chk *chunk.Chunk, colIdx int, gen dataGenerator) {
batchSize := 1024
if gen == nil {
gen = &defaultGener{0.2, eType}
}

Expand Down
74 changes: 27 additions & 47 deletions expression/builtin_like.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,46 +96,32 @@ func (c *regexpFunctionClass) getFunction(ctx sessionctx.Context, args []Express
bf.tp.Flen = 1
var sig builtinFunc
if types.IsBinaryStr(args[0].GetType()) || types.IsBinaryStr(args[1].GetType()) {
sig = &builtinRegexpBinarySig{builtinRegexpSharedSig{baseBuiltinFunc: bf}}
sig = newBuiltinRegexpBinarySig(bf)
} else {
sig = &builtinRegexpSig{builtinRegexpSharedSig{baseBuiltinFunc: bf}}
sig = newBuiltinRegexpSig(bf)
}
return sig, nil
}

type regexpCompiler interface {
compile(pat string) (*regexp.Regexp, error)
}

type builtinRegexpSharedSig struct {
baseBuiltinFunc
compile func(string) (*regexp.Regexp, error)
memoizedRegexp *regexp.Regexp
memoizedErr error
// TODO: By having this struct store a compile() func pointer that is
// customized for builtinRegexpBinarySig and builtinRegexpSig, we can
// remove the regexpCompiler interface and dedupe Clone()/evalInt()/vecEvalInt().
}

func (b *builtinRegexpSharedSig) clone(from *builtinRegexpSharedSig) {
b.cloneFrom(&from.baseBuiltinFunc)
b.compile = from.compile
if from.memoizedRegexp != nil {
b.memoizedRegexp = from.memoizedRegexp.Copy()
}
b.memoizedErr = from.memoizedErr
}

type builtinRegexpBinarySig struct {
builtinRegexpSharedSig
}

func (b *builtinRegexpBinarySig) Clone() builtinFunc {
newSig := &builtinRegexpBinarySig{}
newSig.clone(&b.builtinRegexpSharedSig)
return newSig
}

func (b *builtinRegexpBinarySig) evalInt(row chunk.Row) (int64, bool, error) {
// TODO: this can also be moved into builtinRegexpSharedSig
// evalInt evals `expr REGEXP pat`, or `expr RLIKE pat`.
// See https://dev.mysql.com/doc/refman/5.7/en/regexp.html#operator_regexp
func (b *builtinRegexpSharedSig) evalInt(row chunk.Row) (int64, bool, error) {
expr, isNull, err := b.args[0].EvalString(b.ctx, row)
if isNull || err != nil {
return 0, true, err
Expand All @@ -154,42 +140,36 @@ func (b *builtinRegexpBinarySig) evalInt(row chunk.Row) (int64, bool, error) {
return boolToInt64(re.MatchString(expr)), false, nil
}

func (b *builtinRegexpBinarySig) compile(pat string) (*regexp.Regexp, error) {
return regexp.Compile(pat)
type builtinRegexpBinarySig struct {
builtinRegexpSharedSig
}

type builtinRegexpSig struct {
builtinRegexpSharedSig
func newBuiltinRegexpBinarySig(bf baseBuiltinFunc) *builtinRegexpBinarySig {
shared := builtinRegexpSharedSig{baseBuiltinFunc: bf}
shared.compile = regexp.Compile
return &builtinRegexpBinarySig{builtinRegexpSharedSig: shared}
}

func (b *builtinRegexpSig) Clone() builtinFunc {
newSig := &builtinRegexpSig{}
func (b *builtinRegexpBinarySig) Clone() builtinFunc {
newSig := &builtinRegexpBinarySig{}
newSig.clone(&b.builtinRegexpSharedSig)
return newSig
}

// evalInt evals `expr REGEXP pat`, or `expr RLIKE pat`.
// See https://dev.mysql.com/doc/refman/5.7/en/regexp.html#operator_regexp
func (b *builtinRegexpSig) evalInt(row chunk.Row) (int64, bool, error) {
// TODO: this can also be moved into builtinRegexpSharedSig
expr, isNull, err := b.args[0].EvalString(b.ctx, row)
if isNull || err != nil {
return 0, true, err
}

pat, isNull, err := b.args[1].EvalString(b.ctx, row)
if isNull || err != nil {
return 0, true, err
}
type builtinRegexpSig struct {
builtinRegexpSharedSig
}

// TODO: We don't need to compile pattern if it has been compiled or it is static.
re, err := b.compile(pat)
if err != nil {
return 0, true, ErrRegexp.GenWithStackByArgs(err.Error())
func newBuiltinRegexpSig(bf baseBuiltinFunc) *builtinRegexpSig {
shared := builtinRegexpSharedSig{baseBuiltinFunc: bf}
shared.compile = func(pat string) (*regexp.Regexp, error) {
return regexp.Compile("(?i)" + pat)
}
return boolToInt64(re.MatchString(expr)), false, nil
return &builtinRegexpSig{builtinRegexpSharedSig: shared}
}

func (b *builtinRegexpSig) compile(pat string) (*regexp.Regexp, error) {
return regexp.Compile("(?i)" + pat)
func (b *builtinRegexpSig) Clone() builtinFunc {
newSig := &builtinRegexpSig{}
newSig.clone(&b.builtinRegexpSharedSig)
return newSig
}
18 changes: 5 additions & 13 deletions expression/builtin_like_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,29 +33,21 @@ func (b *builtinRegexpBinarySig) vectorized() bool {
return true
}

func (b *builtinRegexpBinarySig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
return b.builtinRegexpSharedSig.vecEvalInt(input, result, b)
}

func (b *builtinRegexpSig) vectorized() bool {
return true
}

func (b *builtinRegexpSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
return b.builtinRegexpSharedSig.vecEvalInt(input, result, b)
}

func (b *builtinRegexpSharedSig) isMemoizedRegexpInitialized() bool {
return !(b.memoizedRegexp == nil && b.memoizedErr == nil)
}

func (b *builtinRegexpSharedSig) initMemoizedRegexp(patterns *chunk.Column, n int, rc regexpCompiler) {
func (b *builtinRegexpSharedSig) initMemoizedRegexp(patterns *chunk.Column, n int) {
// Precondition: patterns is generated from a constant expression
for i := 0; i < n; i++ {
if patterns.IsNull(i) {
continue
}
re, err := rc.compile(patterns.GetString(i))
re, err := b.compile(patterns.GetString(i))
b.memoizedRegexp = re
b.memoizedErr = err
break
Expand All @@ -68,7 +60,7 @@ func (b *builtinRegexpSharedSig) initMemoizedRegexp(patterns *chunk.Column, n in
}
}

func (b *builtinRegexpSharedSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column, rc regexpCompiler) error {
func (b *builtinRegexpSharedSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
n := input.NumRows()
bufExpr, err := b.bufAllocator.get(types.ETString, n)
if err != nil {
Expand All @@ -89,13 +81,13 @@ func (b *builtinRegexpSharedSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co
}

if b.args[1].ConstItem() && !b.isMemoizedRegexpInitialized() {
b.initMemoizedRegexp(bufPat, n, rc)
b.initMemoizedRegexp(bufPat, n)
}
getRegexp := func(pat string) (*regexp.Regexp, error) {
if b.isMemoizedRegexpInitialized() {
return b.memoizedRegexp, b.memoizedErr
}
return rc.compile(pat)
return b.compile(pat)
}

result.ResizeInt64(n, false)
Expand Down
114 changes: 114 additions & 0 deletions expression/builtin_regexp_vec_const_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
// 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 (
"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"
"github.com/pingcap/tidb/util/testleak"
)

func genVecBuiltinRegexpBenchCaseForConstants() (baseFunc builtinFunc, childrenFieldTypes []*types.FieldType, input *chunk.Chunk, output *chunk.Column) {
const (
numArgs = 2
batchSz = 1024
rePat = `\A[A-Za-z]{3,5}\d{1,5}[[:alpha:]]*\z`
)

childrenFieldTypes = make([]*types.FieldType, numArgs)
for i := 0; i < numArgs; i++ {
childrenFieldTypes[i] = eType2FieldType(types.ETString)
}

input = chunk.New(childrenFieldTypes, batchSz, batchSz)
// Fill the first arg with some random string
fillColumnWithGener(types.ETString, input, 0, &randLenStrGener{lenBegin: 10, lenEnd: 20})
// It seems like we still need to fill this column, otherwise row.GetDatumRow() will crash
fillColumnWithGener(types.ETString, input, 1, &constStrGener{s: rePat})

args := make([]Expression, numArgs)
args[0] = &Column{Index: 0, RetType: childrenFieldTypes[0]}
args[1] = DatumToConstant(types.NewStringDatum(rePat), mysql.TypeString)

var err error
baseFunc, err = funcs[ast.Regexp].getFunction(mock.NewContext(), args)
if err != nil {
panic(err)
}

output = chunk.NewColumn(eType2FieldType(types.ETInt), batchSz)
// Mess up the output to make sure vecEvalXXX to call ResizeXXX/ReserveXXX itself.
output.AppendNull()
return
}

func (s *testEvaluatorSuite) TestVectorizedBuiltinRegexpForConstants(c *C) {
defer testleak.AfterTest(c)()

bf, childrenFieldTypes, input, output := genVecBuiltinRegexpBenchCaseForConstants()
err := bf.vecEvalInt(input, output)
c.Assert(err, IsNil)
i64s := output.Int64s()

it := chunk.NewIterator4Chunk(input)
i := 0
commentf := func(row int) CommentInterface {
return Commentf("func: builtinRegexpSig, row: %v, rowData: %v", row, input.GetRow(row).GetDatumRow(childrenFieldTypes))
}
for row := it.Begin(); row != it.End(); row = it.Next() {
val, isNull, err := bf.evalInt(row)
c.Assert(err, IsNil)
c.Assert(isNull, Equals, output.IsNull(i), commentf(i))
if !isNull {
c.Assert(val, Equals, i64s[i], commentf(i))
}
i++
}
}

func BenchmarkVectorizedBuiltinRegexpForConstants(b *testing.B) {
bf, _, input, output := genVecBuiltinRegexpBenchCaseForConstants()
b.Run("builtinRegexpSig-Constants-VecBuiltinFunc", func(b *testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
if err := bf.vecEvalInt(input, output); err != nil {
b.Fatal(err)
}
}
})
b.Run("builtinRegexpSig-Constants-NonVecBuiltinFunc", func(b *testing.B) {
b.ResetTimer()
it := chunk.NewIterator4Chunk(input)
for i := 0; i < b.N; i++ {
output.Reset()
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := bf.evalInt(row)
if err != nil {
b.Fatal(err)
}
if isNull {
output.AppendNull()
} else {
output.AppendInt64(v)
}
}
}
})
}

0 comments on commit 2d28a27

Please sign in to comment.