From a03d4460a0b23c4e197c4151fe0bb09db5d4833b Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 10 Sep 2019 12:29:06 +0800 Subject: [PATCH 01/15] Modify the vec expression test framework --- expression/bench_test.go | 45 ++++++++++++++------------ expression/builtin_cast.go | 21 ------------ expression/builtin_cast_vec.go | 40 +++++++++++++++++++++++ expression/builtin_cast_vec_test.go | 34 +++++++++++++++++++ expression/builtin_compare_vec_test.go | 34 +++++++++++++++++++ expression/builtin_control_vec_test.go | 34 +++++++++++++++++++ expression/builtin_math_vec_test.go | 34 +++++++++++++++++++ expression/builtin_string_vec_test.go | 34 +++++++++++++++++++ 8 files changed, 234 insertions(+), 42 deletions(-) create mode 100644 expression/builtin_cast_vec.go create mode 100644 expression/builtin_cast_vec_test.go create mode 100644 expression/builtin_compare_vec_test.go create mode 100644 expression/builtin_control_vec_test.go create mode 100644 expression/builtin_math_vec_test.go create mode 100644 expression/builtin_string_vec_test.go diff --git a/expression/bench_test.go b/expression/bench_test.go index f33e906f81522..9448e00d8d21f 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -289,23 +289,26 @@ type vecExprBenchCase struct { geners []dataGenerator } -var vecExprBenchCases = map[string][]vecExprBenchCase{ - ast.Cast: { - {types.ETInt, []types.EvalType{types.ETInt}, nil}, - }, - ast.Repeat: { - {types.ETString, []types.EvalType{types.ETString, types.ETInt}, []dataGenerator{&randLenStrGener{10, 20}, &rangeInt64Gener{-10, 10}}}, - }, - ast.Log10: { - {types.ETReal, []types.EvalType{types.ETReal}, nil}, - }, - ast.If: { - {types.ETJson, []types.EvalType{types.ETInt, types.ETJson, types.ETJson}, nil}, - }, - ast.Greatest: { - {types.ETDecimal, []types.EvalType{types.ETDecimal, types.ETDecimal, types.ETDecimal}, nil}, - }, -} +// +//var vecExprBenchCases = map[string][]vecExprBenchCase{ +// ast.Cast: { +// {types.ETInt, []types.EvalType{types.ETInt}, nil}, +// }, +// ast.Repeat: { +// {types.ETString, []types.EvalType{types.ETString, types.ETInt}, []dataGenerator{&randLenStrGener{10, 20}, &rangeInt64Gener{-10, 10}}}, +// }, +// ast.Log10: { +// {types.ETReal, []types.EvalType{types.ETReal}, nil}, +// }, +// ast.If: { +// {types.ETJson, []types.EvalType{types.ETInt, types.ETJson, types.ETJson}, nil}, +// }, +// ast.Greatest: { +// {types.ETDecimal, []types.EvalType{types.ETDecimal, types.ETDecimal, types.ETDecimal}, nil}, +// }, +//} + +type vecExprBenchCases map[string][]vecExprBenchCase func fillColumn(eType types.EvalType, chk *chunk.Chunk, colIdx int, testCase vecExprBenchCase) { batchSize := 1024 @@ -385,9 +388,9 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx return expr, input, output } -func (s *testEvaluatorSuite) TestVectorizedEvalOneVec(c *C) { +func (s *testEvaluatorSuite) TestVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - for funcName, testCases := range vecExprBenchCases { + for funcName, testCases := range vecExprCases { for _, testCase := range testCases { expr, input, output := genVecExprBenchCase(ctx, funcName, testCase) output2 := output.CopyConstruct() @@ -507,8 +510,8 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } -func (s *testEvaluatorSuite) TestVectorizedBuiltinFunc(c *C) { - for funcName, testCases := range vecExprBenchCases { +func (s *testEvaluatorSuite) TestVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { + for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 87f2a4987c6d9..45e43e341c1da 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -435,10 +435,6 @@ func (b *builtinCastIntAsIntSig) Clone() builtinFunc { return newSig } -func (b *builtinCastIntAsIntSig) vectorized() bool { - return true -} - func (b *builtinCastIntAsIntSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) { res, isNull, err = b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -450,23 +446,6 @@ func (b *builtinCastIntAsIntSig) evalInt(row chunk.Row) (res int64, isNull bool, return } -func (b *builtinCastIntAsIntSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - if err := b.args[0].VecEvalInt(b.ctx, input, result); err != nil { - return err - } - if b.inUnion && mysql.HasUnsignedFlag(b.tp.Flag) { - i64s := result.Int64s() - // the null array of result is set by its child args[0], - // so we can skip it here to make this loop simpler to improve its performance. - for i := range i64s { - if i64s[i] < 0 { - i64s[i] = 0 - } - } - } - return nil -} - type builtinCastIntAsRealSig struct { baseBuiltinCastFunc } diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go new file mode 100644 index 0000000000000..36e489e4956cb --- /dev/null +++ b/expression/builtin_cast_vec.go @@ -0,0 +1,40 @@ +// 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 ( + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/util/chunk" +) + +func (b *builtinCastIntAsIntSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { + if err := b.args[0].VecEvalInt(b.ctx, input, result); err != nil { + return err + } + if b.inUnion && mysql.HasUnsignedFlag(b.tp.Flag) { + i64s := result.Int64s() + // the null array of result is set by its child args[0], + // so we can skip it here to make this loop simpler to improve its performance. + for i := range i64s { + if i64s[i] < 0 { + i64s[i] = 0 + } + } + } + return nil +} + +func (b *builtinCastIntAsIntSig) vectorized() bool { + return true +} diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go new file mode 100644 index 0000000000000..aba260ba988bc --- /dev/null +++ b/expression/builtin_cast_vec_test.go @@ -0,0 +1,34 @@ +// 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 ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/types" +) + +var vecBuiltinCastCases = map[string][]vecExprBenchCase{ + ast.Cast: { + {types.ETInt, []types.EvalType{types.ETInt}, nil}, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinCastEvalOneVec(c *C) { + s.TestVectorizedEvalOneVec(c, vecBuiltinCastCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinCastFunc(c *C) { + s.TestVectorizedBuiltinFunc(c, vecBuiltinCastCases) +} diff --git a/expression/builtin_compare_vec_test.go b/expression/builtin_compare_vec_test.go new file mode 100644 index 0000000000000..f27d5fb7e587c --- /dev/null +++ b/expression/builtin_compare_vec_test.go @@ -0,0 +1,34 @@ +// 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 ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/types" +) + +var vecBuiltinCompareCases = map[string][]vecExprBenchCase{ + ast.Greatest: { + {types.ETDecimal, []types.EvalType{types.ETDecimal, types.ETDecimal, types.ETDecimal}, nil}, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareEvalOneVec(c *C) { + s.TestVectorizedEvalOneVec(c, vecBuiltinCompareCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareFunc(c *C) { + s.TestVectorizedBuiltinFunc(c, vecBuiltinCompareCases) +} diff --git a/expression/builtin_control_vec_test.go b/expression/builtin_control_vec_test.go new file mode 100644 index 0000000000000..54085bffdbabf --- /dev/null +++ b/expression/builtin_control_vec_test.go @@ -0,0 +1,34 @@ +// 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 ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/types" +) + +var vecBuiltinControlCases = map[string][]vecExprBenchCase{ + ast.If: { + {types.ETJson, []types.EvalType{types.ETInt, types.ETJson, types.ETJson}, nil}, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinControlEvalOneVec(c *C) { + s.TestVectorizedEvalOneVec(c, vecBuiltinControlCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinControlFunc(c *C) { + s.TestVectorizedBuiltinFunc(c, vecBuiltinControlCases) +} diff --git a/expression/builtin_math_vec_test.go b/expression/builtin_math_vec_test.go new file mode 100644 index 0000000000000..1e5defe451ae0 --- /dev/null +++ b/expression/builtin_math_vec_test.go @@ -0,0 +1,34 @@ +// 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 ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/types" +) + +var vecBuiltinMathCases = map[string][]vecExprBenchCase{ + ast.Log10: { + {types.ETReal, []types.EvalType{types.ETReal}, nil}, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinMathEvalOneVec(c *C) { + s.TestVectorizedEvalOneVec(c, vecBuiltinMathCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinMathFunc(c *C) { + s.TestVectorizedBuiltinFunc(c, vecBuiltinMathCases) +} diff --git a/expression/builtin_string_vec_test.go b/expression/builtin_string_vec_test.go new file mode 100644 index 0000000000000..bbc761283f3a0 --- /dev/null +++ b/expression/builtin_string_vec_test.go @@ -0,0 +1,34 @@ +// 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 ( + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/types" +) + +var vecBuiltinStringCases = map[string][]vecExprBenchCase{ + ast.Repeat: { + {types.ETString, []types.EvalType{types.ETString, types.ETInt}, []dataGenerator{&randLenStrGener{10, 20}, &rangeInt64Gener{-10, 10}}}, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinStringEvalOneVec(c *C) { + s.TestVectorizedEvalOneVec(c, vecBuiltinStringCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinStringFunc(c *C) { + s.TestVectorizedBuiltinFunc(c, vecBuiltinStringCases) +} From 2aa3a8714446203e06f1a3cff693eabd2df75c3a Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 10 Sep 2019 12:46:48 +0800 Subject: [PATCH 02/15] Modify the vec expression benchmark framework --- expression/bench_test.go | 8 ++++---- expression/builtin_cast_vec_test.go | 10 ++++++++++ expression/builtin_compare_vec_test.go | 10 ++++++++++ expression/builtin_control_vec_test.go | 10 ++++++++++ expression/builtin_math_vec_test.go | 10 ++++++++++ expression/builtin_string_vec_test.go | 10 ++++++++++ 6 files changed, 54 insertions(+), 4 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 9448e00d8d21f..73b281d5dc0b3 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -433,9 +433,9 @@ func (s *testEvaluatorSuite) TestVectorizedEvalOneVec(c *C, vecExprCases vecExpr } } -func BenchmarkVectorizedEvalOneVec(b *testing.B) { +func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - for funcName, testCases := range vecExprBenchCases { + for funcName, testCases := range vecExprCases { for _, testCase := range testCases { expr, input, output := genVecExprBenchCase(ctx, funcName, testCase) exprName := expr.String() @@ -632,9 +632,9 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinFunc(c *C, vecExprCases vecExp } } -func BenchmarkVectorizedBuiltinFunc(b *testing.B) { +func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - for funcName, testCases := range vecExprBenchCases { + for funcName, testCases := range vecExprCases { for _, testCase := range testCases { baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) baseFuncName := fmt.Sprintf("%v", reflect.TypeOf(baseFunc)) diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index aba260ba988bc..da5f5e66a335e 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -14,6 +14,8 @@ package expression import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" @@ -32,3 +34,11 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinCastEvalOneVec(c *C) { func (s *testEvaluatorSuite) TestVectorizedBuiltinCastFunc(c *C) { s.TestVectorizedBuiltinFunc(c, vecBuiltinCastCases) } + +func BenchmarkVectorizedBuiltinCastEvalOneVec(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinCastCases) +} + +func BenchmarkVectorizedBuiltinCastFunc(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinCastCases) +} diff --git a/expression/builtin_compare_vec_test.go b/expression/builtin_compare_vec_test.go index f27d5fb7e587c..6d64df170ba09 100644 --- a/expression/builtin_compare_vec_test.go +++ b/expression/builtin_compare_vec_test.go @@ -14,6 +14,8 @@ package expression import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" @@ -32,3 +34,11 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareEvalOneVec(c *C) { func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareFunc(c *C) { s.TestVectorizedBuiltinFunc(c, vecBuiltinCompareCases) } + +func BenchmarkVectorizedBuiltinCompareEvalOneVec(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinCompareCases) +} + +func BenchmarkVectorizedBuiltinCompareFunc(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinCompareCases) +} diff --git a/expression/builtin_control_vec_test.go b/expression/builtin_control_vec_test.go index 54085bffdbabf..c347aa1434454 100644 --- a/expression/builtin_control_vec_test.go +++ b/expression/builtin_control_vec_test.go @@ -14,6 +14,8 @@ package expression import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" @@ -32,3 +34,11 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinControlEvalOneVec(c *C) { func (s *testEvaluatorSuite) TestVectorizedBuiltinControlFunc(c *C) { s.TestVectorizedBuiltinFunc(c, vecBuiltinControlCases) } + +func BenchmarkVectorizedBuiltinControlEvalOneVec(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinControlCases) +} + +func BenchmarkVectorizedBuiltinControlFunc(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinControlCases) +} diff --git a/expression/builtin_math_vec_test.go b/expression/builtin_math_vec_test.go index 1e5defe451ae0..132e3e27f9654 100644 --- a/expression/builtin_math_vec_test.go +++ b/expression/builtin_math_vec_test.go @@ -14,6 +14,8 @@ package expression import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" @@ -32,3 +34,11 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinMathEvalOneVec(c *C) { func (s *testEvaluatorSuite) TestVectorizedBuiltinMathFunc(c *C) { s.TestVectorizedBuiltinFunc(c, vecBuiltinMathCases) } + +func BenchmarkVectorizedBuiltinMathEvalOneVec(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinMathCases) +} + +func BenchmarkVectorizedBuiltinMathFunc(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinMathCases) +} diff --git a/expression/builtin_string_vec_test.go b/expression/builtin_string_vec_test.go index bbc761283f3a0..482433091fb50 100644 --- a/expression/builtin_string_vec_test.go +++ b/expression/builtin_string_vec_test.go @@ -14,6 +14,8 @@ package expression import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" @@ -32,3 +34,11 @@ func (s *testEvaluatorSuite) TestVectorizedBuiltinStringEvalOneVec(c *C) { func (s *testEvaluatorSuite) TestVectorizedBuiltinStringFunc(c *C) { s.TestVectorizedBuiltinFunc(c, vecBuiltinStringCases) } + +func BenchmarkVectorizedBuiltinStringEvalOneVec(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinStringCases) +} + +func BenchmarkVectorizedBuiltinStringFunc(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinStringCases) +} From 0a588f445168a02c63fc96cd1d62b12176c0fe5f Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 10 Sep 2019 13:02:49 +0800 Subject: [PATCH 03/15] fixup --- expression/bench_test.go | 23 ++--------------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 73b281d5dc0b3..e23fc7ae675f2 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -289,25 +289,6 @@ type vecExprBenchCase struct { geners []dataGenerator } -// -//var vecExprBenchCases = map[string][]vecExprBenchCase{ -// ast.Cast: { -// {types.ETInt, []types.EvalType{types.ETInt}, nil}, -// }, -// ast.Repeat: { -// {types.ETString, []types.EvalType{types.ETString, types.ETInt}, []dataGenerator{&randLenStrGener{10, 20}, &rangeInt64Gener{-10, 10}}}, -// }, -// ast.Log10: { -// {types.ETReal, []types.EvalType{types.ETReal}, nil}, -// }, -// ast.If: { -// {types.ETJson, []types.EvalType{types.ETInt, types.ETJson, types.ETJson}, nil}, -// }, -// ast.Greatest: { -// {types.ETDecimal, []types.EvalType{types.ETDecimal, types.ETDecimal, types.ETDecimal}, nil}, -// }, -//} - type vecExprBenchCases map[string][]vecExprBenchCase func fillColumn(eType types.EvalType, chk *chunk.Chunk, colIdx int, testCase vecExprBenchCase) { @@ -388,7 +369,7 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx return expr, input, output } -func (s *testEvaluatorSuite) TestVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { +func (s *testEvaluatorSuite) testVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() for funcName, testCases := range vecExprCases { for _, testCase := range testCases { @@ -510,7 +491,7 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } -func (s *testEvaluatorSuite) TestVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { +func (s *testEvaluatorSuite) testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() From 3246ac3a34db65de5518ae44984af8d3f572b086 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 10 Sep 2019 13:04:54 +0800 Subject: [PATCH 04/15] fixup --- expression/builtin_cast_vec_test.go | 4 ++-- expression/builtin_compare_vec_test.go | 4 ++-- expression/builtin_control_vec_test.go | 4 ++-- expression/builtin_math_vec_test.go | 4 ++-- expression/builtin_string_vec_test.go | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index da5f5e66a335e..1b4a94f172191 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinCastEvalOneVec(c *C) { - s.TestVectorizedEvalOneVec(c, vecBuiltinCastCases) + s.testVectorizedEvalOneVec(c, vecBuiltinCastCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinCastFunc(c *C) { - s.TestVectorizedBuiltinFunc(c, vecBuiltinCastCases) + s.testVectorizedBuiltinFunc(c, vecBuiltinCastCases) } func BenchmarkVectorizedBuiltinCastEvalOneVec(b *testing.B) { diff --git a/expression/builtin_compare_vec_test.go b/expression/builtin_compare_vec_test.go index 6d64df170ba09..51b2293d1ab91 100644 --- a/expression/builtin_compare_vec_test.go +++ b/expression/builtin_compare_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinCompareCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareEvalOneVec(c *C) { - s.TestVectorizedEvalOneVec(c, vecBuiltinCompareCases) + s.testVectorizedEvalOneVec(c, vecBuiltinCompareCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareFunc(c *C) { - s.TestVectorizedBuiltinFunc(c, vecBuiltinCompareCases) + s.testVectorizedBuiltinFunc(c, vecBuiltinCompareCases) } func BenchmarkVectorizedBuiltinCompareEvalOneVec(b *testing.B) { diff --git a/expression/builtin_control_vec_test.go b/expression/builtin_control_vec_test.go index c347aa1434454..190fc870ea2ed 100644 --- a/expression/builtin_control_vec_test.go +++ b/expression/builtin_control_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinControlCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinControlEvalOneVec(c *C) { - s.TestVectorizedEvalOneVec(c, vecBuiltinControlCases) + s.testVectorizedEvalOneVec(c, vecBuiltinControlCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinControlFunc(c *C) { - s.TestVectorizedBuiltinFunc(c, vecBuiltinControlCases) + s.testVectorizedBuiltinFunc(c, vecBuiltinControlCases) } func BenchmarkVectorizedBuiltinControlEvalOneVec(b *testing.B) { diff --git a/expression/builtin_math_vec_test.go b/expression/builtin_math_vec_test.go index 132e3e27f9654..c1f644cd9b755 100644 --- a/expression/builtin_math_vec_test.go +++ b/expression/builtin_math_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinMathCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinMathEvalOneVec(c *C) { - s.TestVectorizedEvalOneVec(c, vecBuiltinMathCases) + s.testVectorizedEvalOneVec(c, vecBuiltinMathCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinMathFunc(c *C) { - s.TestVectorizedBuiltinFunc(c, vecBuiltinMathCases) + s.testVectorizedBuiltinFunc(c, vecBuiltinMathCases) } func BenchmarkVectorizedBuiltinMathEvalOneVec(b *testing.B) { diff --git a/expression/builtin_string_vec_test.go b/expression/builtin_string_vec_test.go index 482433091fb50..850d330255f39 100644 --- a/expression/builtin_string_vec_test.go +++ b/expression/builtin_string_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinStringCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinStringEvalOneVec(c *C) { - s.TestVectorizedEvalOneVec(c, vecBuiltinStringCases) + s.testVectorizedEvalOneVec(c, vecBuiltinStringCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinStringFunc(c *C) { - s.TestVectorizedBuiltinFunc(c, vecBuiltinStringCases) + s.testVectorizedBuiltinFunc(c, vecBuiltinStringCases) } func BenchmarkVectorizedBuiltinStringEvalOneVec(b *testing.B) { From abcfb5b70a7e6f963001989ea7d444af60eb4859 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 10 Sep 2019 15:58:58 +0800 Subject: [PATCH 05/15] fixup --- expression/bench_test.go | 12 ++++++++++-- expression/builtin_cast_vec_test.go | 4 ++-- expression/builtin_compare_vec_test.go | 4 ++-- expression/builtin_control_vec_test.go | 4 ++-- expression/builtin_math_vec_test.go | 4 ++-- expression/builtin_string_vec_test.go | 4 ++-- 6 files changed, 20 insertions(+), 12 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index e23fc7ae675f2..15f87fa629af4 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -369,7 +369,9 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx return expr, input, output } -func (s *testEvaluatorSuite) testVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { +// testVectorizedEvalOneVec is used to verify that the special vectorized +// expression is evaluated correctly during projection +func testVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() for funcName, testCases := range vecExprCases { for _, testCase := range testCases { @@ -414,6 +416,8 @@ func (s *testEvaluatorSuite) testVectorizedEvalOneVec(c *C, vecExprCases vecExpr } } +// benchmarkVectorizedEvalOneVec is used to get the effect of +// using the special vectorized expression evaluations during projection func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() for funcName, testCases := range vecExprCases { @@ -491,7 +495,9 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } -func (s *testEvaluatorSuite) testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { +// testVectorizedBuiltinFunc is used to verify that the special vectorized +// expression is evaluated correctly +func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() @@ -613,6 +619,8 @@ func (s *testEvaluatorSuite) testVectorizedBuiltinFunc(c *C, vecExprCases vecExp } } +// benchmarkVectorizedBuiltinFunc is used to get the effect of +// using the special vectorized expression evaluations func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() for funcName, testCases := range vecExprCases { diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index 1b4a94f172191..b19bb14f9811d 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinCastEvalOneVec(c *C) { - s.testVectorizedEvalOneVec(c, vecBuiltinCastCases) + testVectorizedEvalOneVec(c, vecBuiltinCastCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinCastFunc(c *C) { - s.testVectorizedBuiltinFunc(c, vecBuiltinCastCases) + testVectorizedBuiltinFunc(c, vecBuiltinCastCases) } func BenchmarkVectorizedBuiltinCastEvalOneVec(b *testing.B) { diff --git a/expression/builtin_compare_vec_test.go b/expression/builtin_compare_vec_test.go index 51b2293d1ab91..924bec0752681 100644 --- a/expression/builtin_compare_vec_test.go +++ b/expression/builtin_compare_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinCompareCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareEvalOneVec(c *C) { - s.testVectorizedEvalOneVec(c, vecBuiltinCompareCases) + testVectorizedEvalOneVec(c, vecBuiltinCompareCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinCompareFunc(c *C) { - s.testVectorizedBuiltinFunc(c, vecBuiltinCompareCases) + testVectorizedBuiltinFunc(c, vecBuiltinCompareCases) } func BenchmarkVectorizedBuiltinCompareEvalOneVec(b *testing.B) { diff --git a/expression/builtin_control_vec_test.go b/expression/builtin_control_vec_test.go index 190fc870ea2ed..8a245027232b9 100644 --- a/expression/builtin_control_vec_test.go +++ b/expression/builtin_control_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinControlCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinControlEvalOneVec(c *C) { - s.testVectorizedEvalOneVec(c, vecBuiltinControlCases) + testVectorizedEvalOneVec(c, vecBuiltinControlCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinControlFunc(c *C) { - s.testVectorizedBuiltinFunc(c, vecBuiltinControlCases) + testVectorizedBuiltinFunc(c, vecBuiltinControlCases) } func BenchmarkVectorizedBuiltinControlEvalOneVec(b *testing.B) { diff --git a/expression/builtin_math_vec_test.go b/expression/builtin_math_vec_test.go index c1f644cd9b755..b4ca038ff2890 100644 --- a/expression/builtin_math_vec_test.go +++ b/expression/builtin_math_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinMathCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinMathEvalOneVec(c *C) { - s.testVectorizedEvalOneVec(c, vecBuiltinMathCases) + testVectorizedEvalOneVec(c, vecBuiltinMathCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinMathFunc(c *C) { - s.testVectorizedBuiltinFunc(c, vecBuiltinMathCases) + testVectorizedBuiltinFunc(c, vecBuiltinMathCases) } func BenchmarkVectorizedBuiltinMathEvalOneVec(b *testing.B) { diff --git a/expression/builtin_string_vec_test.go b/expression/builtin_string_vec_test.go index 850d330255f39..59b14021be63d 100644 --- a/expression/builtin_string_vec_test.go +++ b/expression/builtin_string_vec_test.go @@ -28,11 +28,11 @@ var vecBuiltinStringCases = map[string][]vecExprBenchCase{ } func (s *testEvaluatorSuite) TestVectorizedBuiltinStringEvalOneVec(c *C) { - s.testVectorizedEvalOneVec(c, vecBuiltinStringCases) + testVectorizedEvalOneVec(c, vecBuiltinStringCases) } func (s *testEvaluatorSuite) TestVectorizedBuiltinStringFunc(c *C) { - s.testVectorizedBuiltinFunc(c, vecBuiltinStringCases) + testVectorizedBuiltinFunc(c, vecBuiltinStringCases) } func BenchmarkVectorizedBuiltinStringEvalOneVec(b *testing.B) { From 9d95451d77c250dda025b3dbcbdba36302a8370f Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 11 Sep 2019 15:12:38 +0800 Subject: [PATCH 06/15] Support for testing a single vectorized expression evaluation function signature --- expression/bench_test.go | 29 +++++++++++++++++++++++++---- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 6479762daa520..625dedc5cac3f 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -18,6 +18,7 @@ package expression import ( "fmt" "math/rand" + "os" "reflect" "strings" "testing" @@ -385,7 +386,7 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx return expr, input, output } -// testVectorizedEvalOneVec is used to verify that the special vectorized +// testVectorizedEvalOneVec is used to verify that the vectorized // expression is evaluated correctly during projection func testVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() @@ -433,7 +434,7 @@ func testVectorizedEvalOneVec(c *C, vecExprCases vecExprBenchCases) { } // benchmarkVectorizedEvalOneVec is used to get the effect of -// using the special vectorized expression evaluations during projection +// using the vectorized expression evaluations during projection func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() for funcName, testCases := range vecExprCases { @@ -511,13 +512,27 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } -// testVectorizedBuiltinFunc is used to verify that the special vectorized +// testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { + // If you just want to run a special vectorized expression evaluation. + // You should input "export vec_test_func_sig=XXXsig" first, which "XXXsig" means + // the expression function signature you want to test. + // If you want to test all of them, just ignore it or "export vec_test_func_sig=" + // The following benchmark is also like this. + TestFuncName := os.Getenv("vec_test_func_sig") for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) + baseFuncName := fmt.Sprintf("%v", reflect.TypeOf(baseFunc)) + tmp := strings.Split(baseFuncName, ".") + baseFuncName = tmp[len(tmp)-1] + + if TestFuncName != "" && TestFuncName != baseFuncName { + continue + } + it := chunk.NewIterator4Chunk(input) i := 0 var vecWarnCnt uint16 @@ -636,9 +651,11 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { } // benchmarkVectorizedBuiltinFunc is used to get the effect of -// using the special vectorized expression evaluations +// using the vectorized expression evaluations func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() + + TestFuncName := os.Getenv("vec_test_func_sig") for funcName, testCases := range vecExprCases { for _, testCase := range testCases { baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) @@ -646,6 +663,10 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases tmp := strings.Split(baseFuncName, ".") baseFuncName = tmp[len(tmp)-1] + if TestFuncName != "" && TestFuncName != baseFuncName { + continue + } + b.Run(baseFuncName+"-VecBuiltinFunc", func(b *testing.B) { b.ResetTimer() switch testCase.retEvalType { From ad196a3b1aff04a628faba20fe4444cae0a55367 Mon Sep 17 00:00:00 2001 From: Chengpeng Yan <41809508+Reminiscent@users.noreply.github.com> Date: Wed, 11 Sep 2019 16:42:00 +0800 Subject: [PATCH 07/15] fixup Co-Authored-By: Yuanjia Zhang --- expression/bench_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 625dedc5cac3f..dc410bd2e5c3f 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -515,7 +515,7 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { - // If you just want to run a special vectorized expression evaluation. + // If you just want to run a specified vectorized built-in function. // You should input "export vec_test_func_sig=XXXsig" first, which "XXXsig" means // the expression function signature you want to test. // If you want to test all of them, just ignore it or "export vec_test_func_sig=" From b57a2232b46fd86751d964a6a641cb8d0603caf1 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 18:49:44 +0800 Subject: [PATCH 08/15] fixup --- expression/bench_test.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index c4f9bb1352f50..1baeb4a820e1d 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -16,9 +16,9 @@ package expression // This file contains benchmarks of our expression evaluation. import ( + "flag" "fmt" "math/rand" - "os" "reflect" "strings" "testing" @@ -512,15 +512,14 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } +var vecTestFuncSig = flag.String("vec_test_func_sig", "", + "input the signature you want to test") + // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { - // If you just want to run a special vectorized expression evaluation. - // You should input "export vec_test_func_sig=XXXsig" first, which "XXXsig" means - // the expression function signature you want to test. - // If you want to test all of them, just ignore it or "export vec_test_func_sig=" - // The following benchmark is also like this. - TestFuncName := os.Getenv("vec_test_func_sig") + flag.Parse() + TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() @@ -655,7 +654,8 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - TestFuncName := os.Getenv("vec_test_func_sig") + flag.Parse() + TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) From bfe8701c9d23746b8c2e05e8a0b60594b0eec5e5 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 18:55:05 +0800 Subject: [PATCH 09/15] fixup --- expression/bench_test.go | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index dc410bd2e5c3f..93cae8139c1a9 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -16,9 +16,9 @@ package expression // This file contains benchmarks of our expression evaluation. import ( + "flag" "fmt" "math/rand" - "os" "reflect" "strings" "testing" @@ -512,15 +512,13 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } +var vecTestFuncSig = flag.String("vec_test_func_sig", "", + "input the signature that you want to test.") + // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { - // If you just want to run a specified vectorized built-in function. - // You should input "export vec_test_func_sig=XXXsig" first, which "XXXsig" means - // the expression function signature you want to test. - // If you want to test all of them, just ignore it or "export vec_test_func_sig=" - // The following benchmark is also like this. - TestFuncName := os.Getenv("vec_test_func_sig") + TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() @@ -655,7 +653,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - TestFuncName := os.Getenv("vec_test_func_sig") + TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) From 831d1c57bda4d8b31eb235337e1f299d984e1c1b Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 18:57:01 +0800 Subject: [PATCH 10/15] fixup --- expression/bench_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/expression/bench_test.go b/expression/bench_test.go index 93cae8139c1a9..2ca766f61abfc 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -518,6 +518,7 @@ var vecTestFuncSig = flag.String("vec_test_func_sig", "", // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { + flag.Parse() TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { @@ -653,6 +654,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() + flag.Parse() TestFuncName := *vecTestFuncSig for funcName, testCases := range vecExprCases { for _, testCase := range testCases { From 0ad03322cf3be6a517f256637fc9afc565c36035 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 19:26:29 +0800 Subject: [PATCH 11/15] fixup --- expression/bench_test.go | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 2ca766f61abfc..3413f6407386a 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -512,14 +512,18 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas return baseFunc, input, result } -var vecTestFuncSig = flag.String("vec_test_func_sig", "", - "input the signature that you want to test.") - // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { flag.Parse() - TestFuncName := *vecTestFuncSig + var testFunc map[string]bool + testFunc = make(map[string]bool) + argList := flag.Args() // flag.Args() 返回 -args 后面的所有参数,以切片表示,每个元素代表一个参数 + testAll := true + for _, arg := range argList { + testFunc[arg] = true + testAll = false + } for funcName, testCases := range vecExprCases { for _, testCase := range testCases { ctx := mock.NewContext() @@ -528,7 +532,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { tmp := strings.Split(baseFuncName, ".") baseFuncName = tmp[len(tmp)-1] - if TestFuncName != "" && TestFuncName != baseFuncName { + if !testAll && testFunc[baseFuncName] != true { continue } @@ -655,7 +659,14 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases ctx := mock.NewContext() flag.Parse() - TestFuncName := *vecTestFuncSig + var testFunc map[string]bool + testFunc = make(map[string]bool) + argList := flag.Args() // flag.Args() 返回 -args 后面的所有参数,以切片表示,每个元素代表一个参数 + testAll := true + for _, arg := range argList { + testFunc[arg] = true + testAll = false + } for funcName, testCases := range vecExprCases { for _, testCase := range testCases { baseFunc, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase) @@ -663,7 +674,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases tmp := strings.Split(baseFuncName, ".") baseFuncName = tmp[len(tmp)-1] - if TestFuncName != "" && TestFuncName != baseFuncName { + if !testAll && testFunc[baseFuncName] != true { continue } From 4b9c33c88e3cd8f84010d30d5e353a4ac29ffebe Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 19:28:51 +0800 Subject: [PATCH 12/15] fixup --- expression/bench_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 3413f6407386a..db3115b2a54ca 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -518,7 +518,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { flag.Parse() var testFunc map[string]bool testFunc = make(map[string]bool) - argList := flag.Args() // flag.Args() 返回 -args 后面的所有参数,以切片表示,每个元素代表一个参数 + argList := flag.Args() testAll := true for _, arg := range argList { testFunc[arg] = true @@ -661,7 +661,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases flag.Parse() var testFunc map[string]bool testFunc = make(map[string]bool) - argList := flag.Args() // flag.Args() 返回 -args 后面的所有参数,以切片表示,每个元素代表一个参数 + argList := flag.Args() testAll := true for _, arg := range argList { testFunc[arg] = true From 2162092fb91918a4c2d4141085ed53fdb753c9ad Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Mon, 16 Sep 2019 19:43:22 +0800 Subject: [PATCH 13/15] fixup --- expression/bench_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index e3ea68197e979..eed0e3f0a7022 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -615,7 +615,7 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { c.Assert(err, IsNil) c.Assert(isNull, Equals, output.IsNull(i)) if !isNull { - c.Assert(val, Equals, d64s[i]) + c.Assert(val.Duration, Equals, d64s[i]) } i++ } From 76dc9a520340c536a72f7bba4efe7771f5bf4ed1 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Tue, 17 Sep 2019 09:32:07 +0800 Subject: [PATCH 14/15] fixup --- expression/bench_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index eed0e3f0a7022..a772903573cce 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -528,10 +528,9 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { var testFunc map[string]bool testFunc = make(map[string]bool) argList := flag.Args() - testAll := true + testAll := len(argList) > 0 for _, arg := range argList { testFunc[arg] = true - testAll = false } for funcName, testCases := range vecExprCases { for _, testCase := range testCases { @@ -671,10 +670,9 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases var testFunc map[string]bool testFunc = make(map[string]bool) argList := flag.Args() - testAll := true + testAll := len(argList) > 0 for _, arg := range argList { testFunc[arg] = true - testAll = false } for funcName, testCases := range vecExprCases { for _, testCase := range testCases { From 34dd5942122703350ee8c7f431dc753e646af529 Mon Sep 17 00:00:00 2001 From: Reminiscent Date: Wed, 18 Sep 2019 10:43:40 +0800 Subject: [PATCH 15/15] fixup --- expression/bench_test.go | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 047999cc7b2e6..7faed6a27e34f 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -530,11 +530,9 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { - flag.Parse() - var testFunc map[string]bool - testFunc = make(map[string]bool) + testFunc := make(map[string]bool) argList := flag.Args() - testAll := len(argList) > 0 + testAll := len(argList) == 0 for _, arg := range argList { testFunc[arg] = true } @@ -671,12 +669,9 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { // using the vectorized expression evaluations func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases) { ctx := mock.NewContext() - - flag.Parse() - var testFunc map[string]bool - testFunc = make(map[string]bool) + testFunc := make(map[string]bool) argList := flag.Args() - testAll := len(argList) > 0 + testAll := len(argList) == 0 for _, arg := range argList { testFunc[arg] = true }