diff --git a/.codecov.yml b/.codecov.yml index e52176908cb78..f2482097c10a9 100644 --- a/.codecov.yml +++ b/.codecov.yml @@ -42,4 +42,5 @@ ignore: - "ddl/testutil/.*" - "executor/seqtest/.*" - "metrics/.*" + - "expression/generator/.*" diff --git a/expression/bench_test.go b/expression/bench_test.go index 290b7bbb6e356..f98abb5d852ac 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -449,11 +449,12 @@ func eType2FieldType(eType types.EvalType) *types.FieldType { } func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecExprBenchCase) (expr Expression, fts []*types.FieldType, input *chunk.Chunk, output *chunk.Chunk) { - fts = testCase.childrenFieldTypes - if fts == nil { - fts = make([]*types.FieldType, len(testCase.childrenTypes)) - for i, eType := range testCase.childrenTypes { - fts[i] = eType2FieldType(eType) + fts = make([]*types.FieldType, len(testCase.childrenTypes)) + for i := range fts { + if i < len(testCase.childrenFieldTypes) && testCase.childrenFieldTypes[i] != nil { + fts[i] = testCase.childrenFieldTypes[i] + } else { + fts[i] = eType2FieldType(testCase.childrenTypes[i]) } } cols := make([]Expression, len(testCase.childrenTypes)) @@ -580,11 +581,12 @@ func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases) func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCase vecExprBenchCase) (baseFunc builtinFunc, fts []*types.FieldType, input *chunk.Chunk, result *chunk.Column) { childrenNumber := len(testCase.childrenTypes) - fts = testCase.childrenFieldTypes - if fts == nil { - fts = make([]*types.FieldType, childrenNumber) - for i, eType := range testCase.childrenTypes { - fts[i] = eType2FieldType(eType) + fts = make([]*types.FieldType, childrenNumber) + for i := range fts { + if i < len(testCase.childrenFieldTypes) && testCase.childrenFieldTypes[i] != nil { + fts[i] = testCase.childrenFieldTypes[i] + } else { + fts[i] = eType2FieldType(testCase.childrenTypes[i]) } } cols := make([]Expression, childrenNumber) @@ -622,9 +624,18 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas panic(err) } result = chunk.NewColumn(eType2FieldType(testCase.retEvalType), 1024) + // Mess up the output to make sure vecEvalXXX to call ResizeXXX/ReserveXXX itself. + result.AppendNull() return baseFunc, fts, input, result } +// a hack way to calculate length of a chunk.Column. +func getColumnLen(col *chunk.Column, eType types.EvalType) int { + chk := chunk.New([]*types.FieldType{eType2FieldType(eType)}, 1024, 1024) + chk.SetCol(0, col) + return chk.NumRows() +} + // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { @@ -645,8 +656,10 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { if !testAll && testFunc[baseFuncName] != true { continue } + // do not forget to implement the vectorized method. + c.Assert(baseFunc.vectorized(), IsTrue, Commentf("func: %v", baseFuncName)) commentf := func(row int) CommentInterface { - return Commentf("case %+v, row: %v, rowData: %v", testCase, row, input.GetRow(row).GetDatumRow(fts)) + return Commentf("func: %v, case %+v, row: %v, rowData: %v", baseFuncName, testCase, row, input.GetRow(row).GetDatumRow(fts)) } it := chunk.NewIterator4Chunk(input) i := 0 @@ -655,12 +668,14 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETInt: err := baseFunc.vecEvalInt(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() i64s := output.Int64s() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalInt(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(val, Equals, i64s[i], commentf(i)) } @@ -669,12 +684,14 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETReal: err := baseFunc.vecEvalReal(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() f64s := output.Float64s() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalReal(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(val, Equals, f64s[i], commentf(i)) } @@ -683,12 +700,14 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETDecimal: err := baseFunc.vecEvalDecimal(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() d64s := output.Decimals() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalDecimal(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(*val, Equals, d64s[i], commentf(i)) } @@ -697,12 +716,14 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETDatetime, types.ETTimestamp: err := baseFunc.vecEvalTime(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() t64s := output.Times() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalTime(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(val, Equals, t64s[i], commentf(i)) } @@ -711,12 +732,14 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETDuration: err := baseFunc.vecEvalDuration(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() d64s := output.GoDurations() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalDuration(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(val.Duration, Equals, d64s[i], commentf(i)) } @@ -725,11 +748,13 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETJson: err := baseFunc.vecEvalJSON(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalJSON(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { var cmp int cmp = json.CompareBinary(val, output.GetJSON(i)) @@ -740,11 +765,13 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { case types.ETString: err := baseFunc.vecEvalString(input, output) c.Assert(err, IsNil) + // do not forget to call ResizeXXX/ReserveXXX + c.Assert(getColumnLen(output, testCase.retEvalType), Equals, input.NumRows()) vecWarnCnt = ctx.GetSessionVars().StmtCtx.WarningCount() for row := it.Begin(); row != it.End(); row = it.Next() { val, isNull, err := baseFunc.evalString(row) c.Assert(err, IsNil) - c.Assert(isNull, Equals, output.IsNull(i)) + c.Assert(isNull, Equals, output.IsNull(i), commentf(i)) if !isNull { c.Assert(val, Equals, output.GetString(i), commentf(i)) } diff --git a/expression/builtin.go b/expression/builtin.go index 6839b3d7241a6..896308d70089f 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -16,6 +16,7 @@ // limitations under the License. //go:generate go run generator/control_vec.go +//go:generate go run generator/time_vec.go package expression diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index 2fa60f5ea5df0..74994980fd9b7 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -362,11 +362,38 @@ func (b *builtinCastRealAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul } func (b *builtinCastTimeAsDurationSig) vectorized() bool { - return false + return true } func (b *builtinCastTimeAsDurationSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + arg0, err := b.bufAllocator.get(types.ETDatetime, n) + if err != nil { + return err + } + defer b.bufAllocator.put(arg0) + if err := b.args[0].VecEvalTime(b.ctx, input, arg0); err != nil { + return err + } + arg0s := arg0.Times() + result.ResizeGoDuration(n, false) + result.MergeNulls(arg0) + ds := result.GoDurations() + for i, t := range arg0s { + if result.IsNull(i) { + continue + } + d, err := t.ConvertToDuration() + if err != nil { + return err + } + d, err = d.RoundFrac(int8(b.tp.Decimal)) + if err != nil { + return err + } + ds[i] = d.Duration + } + return nil } func (b *builtinCastDurationAsDurationSig) vectorized() bool { diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index bc7f517808432..0638d0468616d 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -28,9 +28,29 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETDuration, childrenTypes: []types.EvalType{types.ETInt}, geners: []dataGenerator{new(randDurInt)}}, {retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}}, {retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETDecimal}}, + {retEvalType: types.ETDuration, childrenTypes: []types.EvalType{types.ETDatetime}, + geners: []dataGenerator{&dateTimeGenerWithFsp{ + defaultGener: defaultGener{nullRation: 0.2, eType: types.ETDatetime}, + fsp: 1, + }}, + }, }, } +type dateTimeGenerWithFsp struct { + defaultGener + fsp int8 +} + +func (g *dateTimeGenerWithFsp) gen() interface{} { + result := g.defaultGener.gen() + if t, ok := result.(types.Time); ok { + t.Fsp = g.fsp + return t + } + return result +} + func (s *testEvaluatorSuite) TestVectorizedBuiltinCastEvalOneVec(c *C) { testVectorizedEvalOneVec(c, vecBuiltinCastCases) } diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 4e6daf60168fc..e84243221714c 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -220,14 +220,6 @@ func (b *builtinUnixTimestampCurrentSig) vecEvalInt(input *chunk.Chunk, result * return errors.Errorf("not implemented") } -func (b *builtinAddTimeDateTimeNullSig) vectorized() bool { - return false -} - -func (b *builtinAddTimeDateTimeNullSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSubDateIntRealSig) vectorized() bool { return false } @@ -284,14 +276,6 @@ func (b *builtinNowWithArgSig) vecEvalTime(input *chunk.Chunk, result *chunk.Col return errors.Errorf("not implemented") } -func (b *builtinAddDurationAndStringSig) vectorized() bool { - return false -} - -func (b *builtinAddDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSubDateStringRealSig) vectorized() bool { return false } @@ -316,14 +300,6 @@ func (b *builtinSubDateDurationDecimalSig) vecEvalDuration(input *chunk.Chunk, r return errors.Errorf("not implemented") } -func (b *builtinAddDateAndStringSig) vectorized() bool { - return false -} - -func (b *builtinAddDateAndStringSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinNullTimeDiffSig) vectorized() bool { return false } @@ -428,14 +404,6 @@ func (b *builtinAddDateDurationStringSig) vecEvalDuration(input *chunk.Chunk, re return errors.Errorf("not implemented") } -func (b *builtinAddDateAndDurationSig) vectorized() bool { - return false -} - -func (b *builtinAddDateAndDurationSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSubStringAndDurationSig) vectorized() bool { return false } @@ -596,22 +564,6 @@ func (b *builtinTimestampLiteralSig) vecEvalTime(input *chunk.Chunk, result *chu return errors.Errorf("not implemented") } -func (b *builtinAddDatetimeAndDurationSig) vectorized() bool { - return false -} - -func (b *builtinAddDatetimeAndDurationSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - -func (b *builtinAddStringAndDurationSig) vectorized() bool { - return false -} - -func (b *builtinAddStringAndDurationSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinAddDateIntDecimalSig) vectorized() bool { return false } @@ -652,14 +604,6 @@ func (b *builtinAddDateIntRealSig) vecEvalTime(input *chunk.Chunk, result *chunk return errors.Errorf("not implemented") } -func (b *builtinAddDurationAndDurationSig) vectorized() bool { - return false -} - -func (b *builtinAddDurationAndDurationSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSubDurationAndDurationSig) vectorized() bool { return false } @@ -756,14 +700,6 @@ func (b *builtinAddDateDurationRealSig) vecEvalDuration(input *chunk.Chunk, resu return errors.Errorf("not implemented") } -func (b *builtinAddDatetimeAndStringSig) vectorized() bool { - return false -} - -func (b *builtinAddDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSecToTimeSig) vectorized() bool { return false } @@ -964,14 +900,6 @@ func (b *builtinSubDateStringIntSig) vecEvalTime(input *chunk.Chunk, result *chu return errors.Errorf("not implemented") } -func (b *builtinAddTimeStringNullSig) vectorized() bool { - return false -} - -func (b *builtinAddTimeStringNullSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinDateLiteralSig) vectorized() bool { return false } @@ -1004,14 +932,6 @@ func (b *builtinSubDateDurationStringSig) vecEvalDuration(input *chunk.Chunk, re return errors.Errorf("not implemented") } -func (b *builtinAddTimeDurationNullSig) vectorized() bool { - return false -} - -func (b *builtinAddTimeDurationNullSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinSubTimeStringNullSig) vectorized() bool { return false } @@ -1098,14 +1018,6 @@ func (b *builtinCurrentTime1ArgSig) vecEvalDuration(input *chunk.Chunk, result * return errors.Errorf("not implemented") } -func (b *builtinAddStringAndStringSig) vectorized() bool { - return false -} - -func (b *builtinAddStringAndStringSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") -} - func (b *builtinUTCTimestampWithoutArgSig) vectorized() bool { return false } diff --git a/expression/builtin_time_vec_generated.go b/expression/builtin_time_vec_generated.go new file mode 100644 index 0000000000000..116c499f62f18 --- /dev/null +++ b/expression/builtin_time_vec_generated.go @@ -0,0 +1,611 @@ +// 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. + +// Code generated by go generate in expression/generator; DO NOT EDIT. + +package expression + +import ( + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +func (b *builtinAddDatetimeAndDurationSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + if err := b.args[0].VecEvalTime(b.ctx, input, result); err != nil { + return err + } + buf0 := result + + buf1, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalDuration(b.ctx, input, buf1); err != nil { + return err + } + + result.MergeNulls(buf1) + + arg0s := buf0.Times() + + arg1s := buf1.GoDurations() + + resultSlice := result.Times() + + for i := 0; i < n; i++ { + + if result.IsNull(i) { + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := arg1s[i] + + // calculate + + output, err := arg0.Add(b.ctx.GetSessionVars().StmtCtx, types.Duration{Duration: arg1, Fsp: -1}) + if err != nil { + return err + } + + // commit result + + resultSlice[i] = output + + } + return nil +} + +func (b *builtinAddDatetimeAndDurationSig) vectorized() bool { + return true +} + +func (b *builtinAddDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + if err := b.args[0].VecEvalTime(b.ctx, input, result); err != nil { + return err + } + buf0 := result + + buf1, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { + return err + } + + result.MergeNulls(buf1) + + arg0s := buf0.Times() + + resultSlice := result.Times() + + for i := 0; i < n; i++ { + + if result.IsNull(i) { + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := buf1.GetString(i) + + // calculate + + if !isDuration(arg1) { + result.SetNull(i, true) // fixed: true + continue + } + sc := b.ctx.GetSessionVars().StmtCtx + arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.SetNull(i, true) // fixed: true + continue + } + return err + } + + output, err := arg0.Add(sc, arg1Duration) + if err != nil { + return err + } + + // commit result + + resultSlice[i] = output + + } + return nil +} + +func (b *builtinAddDatetimeAndStringSig) vectorized() bool { + return true +} + +func (b *builtinAddDurationAndDurationSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + if err := b.args[0].VecEvalDuration(b.ctx, input, result); err != nil { + return err + } + buf0 := result + + buf1, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalDuration(b.ctx, input, buf1); err != nil { + return err + } + + result.MergeNulls(buf1) + + arg0s := buf0.GoDurations() + + arg1s := buf1.GoDurations() + + resultSlice := result.GoDurations() + + for i := 0; i < n; i++ { + + if result.IsNull(i) { + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := arg1s[i] + + // calculate + + output, err := types.AddDuration(arg0, arg1) + if err != nil { + return err + } + + // commit result + + resultSlice[i] = output + + } + return nil +} + +func (b *builtinAddDurationAndDurationSig) vectorized() bool { + return true +} + +func (b *builtinAddDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + if err := b.args[0].VecEvalDuration(b.ctx, input, result); err != nil { + return err + } + buf0 := result + + buf1, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { + return err + } + + result.MergeNulls(buf1) + + arg0s := buf0.GoDurations() + + resultSlice := result.GoDurations() + + for i := 0; i < n; i++ { + + if result.IsNull(i) { + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := buf1.GetString(i) + + // calculate + + if !isDuration(arg1) { + result.SetNull(i, true) // fixed: true + continue + } + sc := b.ctx.GetSessionVars().StmtCtx + arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.SetNull(i, true) // fixed: true + continue + } + return err + } + + output, err := types.AddDuration(arg0, arg1Duration.Duration) + if err != nil { + return err + } + + // commit result + + resultSlice[i] = output + + } + return nil +} + +func (b *builtinAddDurationAndStringSig) vectorized() bool { + return true +} + +func (b *builtinAddStringAndDurationSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + buf0, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEvalString(b.ctx, input, buf0); err != nil { + return err + } + + buf1, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalDuration(b.ctx, input, buf1); err != nil { + return err + } + + result.ReserveString(n) + + arg1s := buf1.GoDurations() + + for i := 0; i < n; i++ { + + if buf0.IsNull(i) || buf1.IsNull(i) { + result.AppendNull() + continue + } + + // get arg0 & arg1 + + arg0 := buf0.GetString(i) + + arg1 := arg1s[i] + + // calculate + + sc := b.ctx.GetSessionVars().StmtCtx + fsp1 := int8(b.args[1].GetType().Decimal) + arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} + + var output string + if isDuration(arg0) { + output, err = strDurationAddDuration(sc, arg0, arg1Duration) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.AppendNull() // fixed: false + continue + } + return err + } + } else { + output, err = strDatetimeAddDuration(sc, arg0, arg1Duration) + if err != nil { + return err + } + } + + // commit result + + result.AppendString(output) + + } + return nil +} + +func (b *builtinAddStringAndDurationSig) vectorized() bool { + return true +} + +func (b *builtinAddStringAndStringSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + buf0, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEvalString(b.ctx, input, buf0); err != nil { + return err + } + + arg1Type := b.args[1].GetType() + if mysql.HasBinaryFlag(arg1Type.Flag) { + result.ReserveString(n) + for i := 0; i < n; i++ { + result.AppendNull() + } + return nil + } + + buf1, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { + return err + } + + result.ReserveString(n) + + for i := 0; i < n; i++ { + + if buf0.IsNull(i) || buf1.IsNull(i) { + result.AppendNull() + continue + } + + // get arg0 & arg1 + + arg0 := buf0.GetString(i) + + arg1 := buf1.GetString(i) + + // calculate + + sc := b.ctx.GetSessionVars().StmtCtx + arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.AppendNull() // fixed: false + continue + } + return err + } + + var output string + if isDuration(arg0) { + output, err = strDurationAddDuration(sc, arg0, arg1Duration) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.AppendNull() // fixed: false + continue + } + return err + } + } else { + output, err = strDatetimeAddDuration(sc, arg0, arg1Duration) + if err != nil { + return err + } + } + + // commit result + + result.AppendString(output) + + } + return nil +} + +func (b *builtinAddStringAndStringSig) vectorized() bool { + return true +} + +func (b *builtinAddDateAndDurationSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + buf0, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEvalDuration(b.ctx, input, buf0); err != nil { + return err + } + + buf1, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalDuration(b.ctx, input, buf1); err != nil { + return err + } + + result.ReserveString(n) + + arg0s := buf0.GoDurations() + + arg1s := buf1.GoDurations() + + for i := 0; i < n; i++ { + + if buf0.IsNull(i) || buf1.IsNull(i) { + result.AppendNull() + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := arg1s[i] + + // calculate + + fsp0 := int8(b.args[0].GetType().Decimal) + fsp1 := int8(b.args[1].GetType().Decimal) + arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} + sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) + if err != nil { + return err + } + output := sum.String() + + // commit result + + result.AppendString(output) + + } + return nil +} + +func (b *builtinAddDateAndDurationSig) vectorized() bool { + return true +} + +func (b *builtinAddDateAndStringSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + buf0, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEvalDuration(b.ctx, input, buf0); err != nil { + return err + } + + buf1, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { + return err + } + + result.ReserveString(n) + + arg0s := buf0.GoDurations() + + for i := 0; i < n; i++ { + + if buf0.IsNull(i) || buf1.IsNull(i) { + result.AppendNull() + continue + } + + // get arg0 & arg1 + + arg0 := arg0s[i] + + arg1 := buf1.GetString(i) + + // calculate + + if !isDuration(arg1) { + result.AppendNull() // fixed: false + continue + } + sc := b.ctx.GetSessionVars().StmtCtx + arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + result.AppendNull() // fixed: false + continue + } + return err + } + + fsp0 := int8(b.args[0].GetType().Decimal) + sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) + if err != nil { + return err + } + output := sum.String() + + // commit result + + result.AppendString(output) + + } + return nil +} + +func (b *builtinAddDateAndStringSig) vectorized() bool { + return true +} + +func (b *builtinAddTimeDateTimeNullSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + result.ResizeTime(n, true) + + return nil +} + +func (b *builtinAddTimeDateTimeNullSig) vectorized() bool { + return true +} + +func (b *builtinAddTimeStringNullSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + result.ReserveString(n) + for i := 0; i < n; i++ { + result.AppendNull() + } + + return nil +} + +func (b *builtinAddTimeStringNullSig) vectorized() bool { + return true +} + +func (b *builtinAddTimeDurationNullSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + + result.ResizeGoDuration(n, true) + + return nil +} + +func (b *builtinAddTimeDurationNullSig) vectorized() bool { + return true +} diff --git a/expression/builtin_time_vec_generated_test.go b/expression/builtin_time_vec_generated_test.go new file mode 100644 index 0000000000000..0ff56fe9422cf --- /dev/null +++ b/expression/builtin_time_vec_generated_test.go @@ -0,0 +1,182 @@ +// 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. + +// Code generated by go generate in expression/generator; DO NOT EDIT. + +package expression + +import ( + "testing" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +type gener struct { + defaultGener +} + +func (g gener) gen() interface{} { + result := g.defaultGener.gen() + if _, ok := result.(string); ok { + dg := &defaultGener{eType: types.ETDuration, nullRation: 0} + d := dg.gen().(types.Duration) + if int8(d.Duration)%2 == 0 { + d.Fsp = 0 + } else { + d.Fsp = 1 + } + result = d.String() + } + return result +} + +var vecBuiltinTimeGeneratedCases = map[string][]vecExprBenchCase{ + + ast.AddTime: { + // builtinAddDatetimeAndDurationSig + { + retEvalType: types.ETDatetime, + childrenTypes: []types.EvalType{types.ETDatetime, types.ETDuration}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + }, + }, + // builtinAddDatetimeAndStringSig + { + retEvalType: types.ETDatetime, + childrenTypes: []types.EvalType{types.ETDatetime, types.ETString}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + }, + }, + // builtinAddDurationAndDurationSig + { + retEvalType: types.ETDuration, + childrenTypes: []types.EvalType{types.ETDuration, types.ETDuration}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + }, + }, + // builtinAddDurationAndStringSig + { + retEvalType: types.ETDuration, + childrenTypes: []types.EvalType{types.ETDuration, types.ETString}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + }, + }, + // builtinAddStringAndDurationSig + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETString, types.ETDuration}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + }, + }, + // builtinAddStringAndStringSig + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETString, types.ETString}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + }, + }, + // builtinAddDateAndDurationSig + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETDuration, types.ETDuration}, + + childrenFieldTypes: []*types.FieldType{types.NewFieldType(mysql.TypeDate), types.NewFieldType(mysql.TypeDuration)}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + }, + }, + // builtinAddDateAndStringSig + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETDuration, types.ETString}, + + childrenFieldTypes: []*types.FieldType{types.NewFieldType(mysql.TypeDate), types.NewFieldType(mysql.TypeString)}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + }, + }, + // builtinAddTimeDateTimeNullSig + { + retEvalType: types.ETDatetime, + childrenTypes: []types.EvalType{types.ETDatetime, types.ETDatetime}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + }, + }, + // builtinAddTimeStringNullSig + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETDatetime, types.ETDatetime}, + + childrenFieldTypes: []*types.FieldType{types.NewFieldType(mysql.TypeDate), types.NewFieldType(mysql.TypeDatetime)}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + }, + }, + // builtinAddTimeDurationNullSig + { + retEvalType: types.ETDuration, + childrenTypes: []types.EvalType{types.ETDuration, types.ETDatetime}, + + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETDuration, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETDatetime, nullRation: 0.2}}, + }, + }, + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinTimeEvalOneVecGenerated(c *C) { + testVectorizedEvalOneVec(c, vecBuiltinTimeGeneratedCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltinTimeFuncGenerated(c *C) { + testVectorizedBuiltinFunc(c, vecBuiltinTimeGeneratedCases) +} + +func BenchmarkVectorizedBuiltinTimeEvalOneVecGenerated(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltinTimeGeneratedCases) +} + +func BenchmarkVectorizedBuiltinTimeFuncGenerated(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltinTimeGeneratedCases) +} diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index ddf09ea4df361..6f1de4730582e 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -52,7 +52,24 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ ast.SubDate: {}, ast.AddDate: {}, ast.SubTime: {}, - ast.AddTime: {}, + ast.AddTime: { + // builtinAddStringAndStringSig, a special case written by hand. + // arg1 has BinaryFlag here. + { + retEvalType: types.ETString, + childrenTypes: []types.EvalType{types.ETString, types.ETString}, + childrenFieldTypes: []*types.FieldType{nil, { + Tp: mysql.TypeString, + Flen: types.UnspecifiedLength, + Decimal: types.UnspecifiedLength, + Flag: mysql.BinaryFlag, + }}, + geners: []dataGenerator{ + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + gener{defaultGener{eType: types.ETString, nullRation: 0.2}}, + }, + }, + }, ast.Month: { {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDatetime}}, }, diff --git a/expression/generator/helper/helper.go b/expression/generator/helper/helper.go new file mode 100644 index 0000000000000..934e21995c8f3 --- /dev/null +++ b/expression/generator/helper/helper.go @@ -0,0 +1,46 @@ +// 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 helper + +// TypeContext is the template context for each "github.com/pingcap/tidb/types".EvalType . +type TypeContext struct { + // Describe the name of "github.com/pingcap/tidb/types".ET{{ .ETName }} . + ETName string + // Describe the name of "github.com/pingcap/tidb/expression".VecExpr.VecEval{{ .TypeName }} . + TypeName string + // Describe the name of "github.com/pingcap/tidb/util/chunk".*Column.Append{{ .TypeNameInColumn }}, + // Resize{{ .TypeNameInColumn }}, Reserve{{ .TypeNameInColumn }}, Get{{ .TypeNameInColumn }} and + // {{ .TypeNameInColumn }}s. + // If undefined, it's same as TypeName. + TypeNameInColumn string + // Same as "github.com/pingcap/tidb/util/chunk".getFixedLen() . + Fixed bool +} + +var ( + // TypeInt represents the template context of types.ETInt . + TypeInt = TypeContext{ETName: "Int", TypeName: "Int", TypeNameInColumn: "Int64", Fixed: true} + // TypeReal represents the template context of types.ETReal . + TypeReal = TypeContext{ETName: "Real", TypeName: "Real", TypeNameInColumn: "Float64", Fixed: true} + // TypeDecimal represents the template context of types.ETDecimal . + TypeDecimal = TypeContext{ETName: "Decimal", TypeName: "Decimal", TypeNameInColumn: "Decimal", Fixed: true} + // TypeString represents the template context of types.ETString . + TypeString = TypeContext{ETName: "String", TypeName: "String", TypeNameInColumn: "String", Fixed: false} + // TypeDatetime represents the template context of types.ETDatetime . + TypeDatetime = TypeContext{ETName: "Datetime", TypeName: "Time", TypeNameInColumn: "Time", Fixed: true} + // TypeDuration represents the template context of types.ETDuration . + TypeDuration = TypeContext{ETName: "Duration", TypeName: "Duration", TypeNameInColumn: "GoDuration", Fixed: true} + // TypeJSON represents the template context of types.ETJson . + TypeJSON = TypeContext{ETName: "Json", TypeName: "JSON", TypeNameInColumn: "JSON", Fixed: false} +) diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go new file mode 100644 index 0000000000000..86f413f1cd2ff --- /dev/null +++ b/expression/generator/time_vec.go @@ -0,0 +1,420 @@ +// 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. + +// +build ignore + +package main + +import ( + "bytes" + "flag" + "go/format" + "io/ioutil" + "log" + "path/filepath" + "text/template" + + . "github.com/pingcap/tidb/expression/generator/helper" +) + +var addTime = template.Must(template.New("").Parse(`// 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. + +// Code generated by go generate in expression/generator; DO NOT EDIT. + +package expression + +import ( + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +{{ define "SetNull" }}{{if .Output.Fixed}}result.SetNull(i, true){{else}}result.AppendNull(){{end}} // fixed: {{.Output.Fixed }}{{ end }} +{{ define "ConvertStringToDuration" }} + {{ if ne .SigName "builtinAddStringAndStringSig" }} + if !isDuration(arg1) { + {{ template "SetNull" . }} + continue + }{{ end }} + sc := b.ctx.GetSessionVars().StmtCtx + arg1Duration, err := types.ParseDuration(sc, arg1, {{if eq .Output.TypeName "String"}}getFsp4TimeAddSub{{else}}types.GetFsp{{end}}(arg1)) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + {{ template "SetNull" . }} + continue + } + return err + } +{{ end }} +{{ define "strDurationAddDuration" }} + var output string + if isDuration(arg0) { + output, err = strDurationAddDuration(sc, arg0, arg1Duration) + if err != nil { + if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { + sc.AppendWarning(err) + {{ template "SetNull" . }} + continue + } + return err + } + } else { + output, err = strDatetimeAddDuration(sc, arg0, arg1Duration) + if err != nil { + return err + } + } +{{ end }} + +{{ range . }} +{{ if .AllNull}} +func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + {{ if .Output.Fixed }} + result.Resize{{ .Output.TypeNameInColumn }}(n, true) + {{ else }} + result.Reserve{{ .Output.TypeNameInColumn }}(n) + for i := 0; i < n; i++ { result.AppendNull() } + {{ end }} + return nil +} +{{ else }} +func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() +{{ $reuse := (and (eq .TypeA.TypeName .Output.TypeName) .TypeA.Fixed) }} +{{ if $reuse }} + if err := b.args[0].VecEval{{ .TypeA.TypeName }}(b.ctx, input, result); err != nil { + return err + } + buf0 := result +{{ else }} + buf0, err := b.bufAllocator.get(types.ET{{.TypeA.ETName}}, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err := b.args[0].VecEval{{ .TypeA.TypeName }}(b.ctx, input, buf0); err != nil { + return err + } +{{ end }} + +{{ if eq .SigName "builtinAddStringAndStringSig" }} + arg1Type := b.args[1].GetType() + if mysql.HasBinaryFlag(arg1Type.Flag) { + result.Reserve{{ .Output.TypeNameInColumn }}(n) + for i := 0; i < n; i++ { + result.AppendNull() + } + return nil + } +{{ end }} + + buf1, err := b.bufAllocator.get(types.ET{{.TypeB.ETName}}, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err := b.args[1].VecEval{{ .TypeB.TypeName }}(b.ctx, input, buf1); err != nil { + return err + } + +{{ if $reuse }} + result.MergeNulls(buf1) +{{ else if .Output.Fixed}} + result.Resize{{ .Output.TypeNameInColumn }}(n, false) + result.MergeNulls(buf0, buf1) +{{ else }} + result.Reserve{{ .Output.TypeNameInColumn}}(n) +{{ end }} + +{{ if .TypeA.Fixed }} + arg0s := buf0.{{.TypeA.TypeNameInColumn}}s() +{{ end }} +{{ if .TypeB.Fixed }} + arg1s := buf1.{{.TypeB.TypeNameInColumn}}s() +{{ end }} +{{ if .Output.Fixed }} + resultSlice := result.{{.Output.TypeNameInColumn}}s() +{{ end }} + for i := 0; i < n; i++ { + {{ if .Output.Fixed }} + if result.IsNull(i) { + continue + } + {{ else }} + if buf0.IsNull(i) || buf1.IsNull(i) { + result.AppendNull() + continue + } + {{ end }} + + // get arg0 & arg1 + {{ if .TypeA.Fixed }} + arg0 := arg0s[i] + {{ else }} + arg0 := buf0.Get{{ .TypeA.TypeNameInColumn }}(i) + {{ end }} + {{ if .TypeB.Fixed }} + arg1 := arg1s[i] + {{ else }} + arg1 := buf1.Get{{ .TypeB.TypeNameInColumn }}(i) + {{ end }} + + // calculate + {{ if eq .SigName "builtinAddDatetimeAndDurationSig" }} + output, err := arg0.Add(b.ctx.GetSessionVars().StmtCtx, types.Duration{Duration: arg1, Fsp: -1}) + if err != nil { + return err + } + {{ else if eq .SigName "builtinAddDatetimeAndStringSig" }} + {{ template "ConvertStringToDuration" . }} + output, err := arg0.Add(sc, arg1Duration) + if err != nil { + return err + } + {{ else if eq .SigName "builtinAddDurationAndDurationSig" }} + output, err := types.AddDuration(arg0, arg1) + if err != nil { + return err + } + {{ else if eq .SigName "builtinAddDurationAndStringSig" }} + {{ template "ConvertStringToDuration" . }} + output, err := types.AddDuration(arg0, arg1Duration.Duration) + if err != nil { + return err + } + {{ else if eq .SigName "builtinAddStringAndDurationSig" }} + sc := b.ctx.GetSessionVars().StmtCtx + fsp1 := int8(b.args[1].GetType().Decimal) + arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} + {{ template "strDurationAddDuration" . }} + {{ else if eq .SigName "builtinAddStringAndStringSig" }} + {{ template "ConvertStringToDuration" . }} + {{ template "strDurationAddDuration" . }} + {{ else if eq .SigName "builtinAddDateAndDurationSig" }} + fsp0 := int8(b.args[0].GetType().Decimal) + fsp1 := int8(b.args[1].GetType().Decimal) + arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} + sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) + if err != nil { + return err + } + output := sum.String() + {{ else if eq .SigName "builtinAddDateAndStringSig" }} + {{ template "ConvertStringToDuration" . }} + fsp0 := int8(b.args[0].GetType().Decimal) + sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) + if err != nil { + return err + } + output := sum.String() + {{ end }} + + // commit result + {{ if .Output.Fixed }} + resultSlice[i] = output + {{ else }} + result.Append{{ .Output.TypeNameInColumn }}(output) + {{ end }} + } + return nil +} +{{ end }}{{/* if .AllNull */}} + +func (b *{{.SigName}}) vectorized() bool { + return true +} +{{ end }}{{/* range */}} +`)) + +var testFile = template.Must(template.New("").Parse(`// 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. + +// Code generated by go generate in expression/generator; DO NOT EDIT. + +package expression + +import ( + "testing" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +type gener struct { + defaultGener +} + +func (g gener) gen() interface{} { + result := g.defaultGener.gen() + if _, ok := result.(string); ok { + dg := &defaultGener{eType: types.ETDuration, nullRation: 0} + d := dg.gen().(types.Duration) + if int8(d.Duration)%2 == 0 { + d.Fsp = 0 + } else { + d.Fsp = 1 + } + result = d.String() + } + return result +} + +{{/* Add more test cases here if we have more functions in this file */}} +var vecBuiltin{{.Category}}GeneratedCases = map[string][]vecExprBenchCase{ +{{ range .Functions }} + ast.{{.FuncName}}: { + {{ range .Sigs }} // {{ .SigName }} + { + retEvalType: types.ET{{ .Output.ETName }}, + childrenTypes: []types.EvalType{types.ET{{ .TypeA.ETName }}, types.ET{{ .TypeB.ETName }}}, + {{ if ne .FieldTypeA "" }} + childrenFieldTypes: []*types.FieldType{types.NewFieldType(mysql.Type{{.FieldTypeA}}), types.NewFieldType(mysql.Type{{.FieldTypeB}})}, + {{ end }} + geners: []dataGenerator{ + gener{defaultGener{eType: types.ET{{.TypeA.ETName}}, nullRation: 0.2}}, + gener{defaultGener{eType: types.ET{{.TypeB.ETName}}, nullRation: 0.2}}, + }, + }, + {{ end }} +{{ end }} + }, +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltin{{.Category}}EvalOneVecGenerated(c *C) { + testVectorizedEvalOneVec(c, vecBuiltin{{.Category}}GeneratedCases) +} + +func (s *testEvaluatorSuite) TestVectorizedBuiltin{{.Category}}FuncGenerated(c *C) { + testVectorizedBuiltinFunc(c, vecBuiltin{{.Category}}GeneratedCases) +} + +func BenchmarkVectorizedBuiltin{{.Category}}EvalOneVecGenerated(b *testing.B) { + benchmarkVectorizedEvalOneVec(b, vecBuiltin{{.Category}}GeneratedCases) +} + +func BenchmarkVectorizedBuiltin{{.Category}}FuncGenerated(b *testing.B) { + benchmarkVectorizedBuiltinFunc(b, vecBuiltin{{.Category}}GeneratedCases) +} +`)) + +var addTimeSigsTmpl = []sig{ + {SigName: "builtinAddDatetimeAndDurationSig", TypeA: TypeDatetime, TypeB: TypeDuration, Output: TypeDatetime}, + {SigName: "builtinAddDatetimeAndStringSig", TypeA: TypeDatetime, TypeB: TypeString, Output: TypeDatetime}, + {SigName: "builtinAddDurationAndDurationSig", TypeA: TypeDuration, TypeB: TypeDuration, Output: TypeDuration}, + {SigName: "builtinAddDurationAndStringSig", TypeA: TypeDuration, TypeB: TypeString, Output: TypeDuration}, + {SigName: "builtinAddStringAndDurationSig", TypeA: TypeString, TypeB: TypeDuration, Output: TypeString}, + {SigName: "builtinAddStringAndStringSig", TypeA: TypeString, TypeB: TypeString, Output: TypeString}, + {SigName: "builtinAddDateAndDurationSig", TypeA: TypeDuration, TypeB: TypeDuration, Output: TypeString, FieldTypeA: "Date", FieldTypeB: "Duration"}, + {SigName: "builtinAddDateAndStringSig", TypeA: TypeDuration, TypeB: TypeString, Output: TypeString, FieldTypeA: "Date", FieldTypeB: "String"}, + + {SigName: "builtinAddTimeDateTimeNullSig", TypeA: TypeDatetime, TypeB: TypeDatetime, Output: TypeDatetime, AllNull: true}, + {SigName: "builtinAddTimeStringNullSig", TypeA: TypeDatetime, TypeB: TypeDatetime, Output: TypeString, AllNull: true, FieldTypeA: "Date", FieldTypeB: "Datetime"}, + {SigName: "builtinAddTimeDurationNullSig", TypeA: TypeDuration, TypeB: TypeDatetime, Output: TypeDuration, AllNull: true}, +} + +type sig struct { + SigName string + TypeA, TypeB, Output TypeContext + FieldTypeA, FieldTypeB string // Optional + AllNull bool +} + +type function struct { + FuncName string + Sigs []sig +} + +var tmplVal = struct { + Category string + Functions []function +}{ + Category: "Time", + Functions: []function{ + {FuncName: "AddTime", Sigs: addTimeSigsTmpl}, + }, +} + +func generateDotGo(fileName string) error { + w := new(bytes.Buffer) + err := addTime.Execute(w, addTimeSigsTmpl) + if err != nil { + return err + } + data, err := format.Source(w.Bytes()) + if err != nil { + log.Println("[Warn]", fileName+": gofmt failed", err) + data = w.Bytes() // write original data for debugging + } + return ioutil.WriteFile(fileName, data, 0644) +} + +func generateTestDotGo(fileName string) error { + w := new(bytes.Buffer) + err := testFile.Execute(w, tmplVal) + if err != nil { + return err + } + data, err := format.Source(w.Bytes()) + if err != nil { + log.Println("[Warn]", fileName+": gofmt failed", err) + data = w.Bytes() // write original data for debugging + } + return ioutil.WriteFile(fileName, data, 0644) +} + +// generateOneFile generate one xxx.go file and the associated xxx_test.go file. +func generateOneFile(fileNamePrefix string) (err error) { + + err = generateDotGo(fileNamePrefix + ".go") + if err != nil { + return + } + err = generateTestDotGo(fileNamePrefix + "_test.go") + return +} + +func main() { + flag.Parse() + var err error + outputDir := "." + err = generateOneFile(filepath.Join(outputDir, "builtin_time_vec_generated")) + if err != nil { + log.Fatalln("generateOneFile", err) + } +} diff --git a/types/overflow.go b/types/overflow.go index 6253357cbe047..07bdc789e528b 100644 --- a/types/overflow.go +++ b/types/overflow.go @@ -16,6 +16,7 @@ package types import ( "fmt" "math" + "time" "github.com/pingcap/errors" ) @@ -38,6 +39,16 @@ func AddInt64(a int64, b int64) (int64, error) { return a + b, nil } +// AddDuration adds time.Duration a and b if no overflow, otherwise returns error. +func AddDuration(a time.Duration, b time.Duration) (time.Duration, error) { + if (a > 0 && b > 0 && math.MaxInt64-a < b) || + (a < 0 && b < 0 && math.MinInt64-a > b) { + return 0, ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%d, %d)", int64(a), int64(b))) + } + + return a + b, nil +} + // AddInteger adds uint64 a and int64 b and returns uint64 if no overflow error. func AddInteger(a uint64, b int64) (uint64, error) { if b >= 0 { diff --git a/types/overflow_test.go b/types/overflow_test.go index 43d6d84a88c9a..27b7c2f80dd70 100644 --- a/types/overflow_test.go +++ b/types/overflow_test.go @@ -15,6 +15,7 @@ package types import ( "math" + "time" . "github.com/pingcap/check" "github.com/pingcap/tidb/util/testleak" @@ -69,6 +70,12 @@ func (s *testOverflowSuite) TestAdd(c *C) { } else { c.Assert(ret, Equals, t.ret) } + ret2, err := AddDuration(time.Duration(t.lsh), time.Duration(t.rsh)) + if t.overflow { + c.Assert(err, NotNil) + } else { + c.Assert(ret2, Equals, time.Duration(t.ret)) + } } tblInt := []struct { diff --git a/util/chunk/column.go b/util/chunk/column.go index c824fbd9c495b..06ee13c734592 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -645,8 +645,8 @@ func (c *Column) CopyReconstruct(sel []int, dst *Column) *Column { // MergeNulls merges these columns' null bitmaps. // For a row, if any column of it is null, the result is null. // It works like: if col1.IsNull || col2.IsNull || col3.IsNull. -// The user should ensure that all these columns have the same length, and -// data stored in these columns are fixed-length type. +// The caller should ensure that all these columns have the same +// length, and data stored in the result column is fixed-length type. func (c *Column) MergeNulls(cols ...*Column) { for _, col := range cols { for i := range c.nullBitmap {