diff --git a/expression/bench_test.go b/expression/bench_test.go index d31493ef6c8f0..57690a68c86ed 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -1097,6 +1097,9 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { AuthUsername: "tidb", } } + if funcName == ast.Sleep { + ctx.GetSessionVars().StrictSQLMode = false + } if funcName == ast.GetParam { testTime := time.Now() ctx.GetSessionVars().PreparedParams = []types.Datum{ @@ -1317,6 +1320,9 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases AuthUsername: "tidb", } } + if funcName == ast.Sleep { + ctx.GetSessionVars().StrictSQLMode = false + } if funcName == ast.GetParam { testTime := time.Now() ctx.GetSessionVars().PreparedParams = []types.Datum{ diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index 13943339cf159..61112b8aa2089 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -25,6 +25,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" @@ -108,6 +109,24 @@ func (b *builtinSleepSig) Clone() builtinFunc { return newSig } +func sleepDuration(sessVars *variable.SessionVars, val float64) (killed bool) { + dur := time.Duration(val * float64(time.Second.Nanoseconds())) + ticker := time.NewTicker(10 * time.Millisecond) + defer ticker.Stop() + start := time.Now() + finish := false + for !finish { + now := <-ticker.C + if now.Sub(start) > dur { + finish = true + } + if atomic.CompareAndSwapUint32(&sessVars.Killed, 1, 0) { + return true + } + } + return false +} + // evalInt evals a builtinSleepSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_sleep func (b *builtinSleepSig) evalInt(row chunk.Row) (int64, bool, error) { @@ -135,22 +154,9 @@ func (b *builtinSleepSig) evalInt(row chunk.Row) (int64, bool, error) { return 0, false, errIncorrectArgs.GenWithStackByArgs("sleep") } - dur := time.Duration(val * float64(time.Second.Nanoseconds())) - ticker := time.NewTicker(10 * time.Millisecond) - defer ticker.Stop() - start := time.Now() - finish := false - for !finish { - select { - case now := <-ticker.C: - if now.Sub(start) > dur { - finish = true - } - default: - if atomic.CompareAndSwapUint32(&sessVars.Killed, 1, 0) { - return 1, false, nil - } - } + flag := sleepDuration(sessVars, val) + if flag { + return 1, false, nil } return 0, false, nil diff --git a/expression/builtin_miscellaneous_vec.go b/expression/builtin_miscellaneous_vec.go index 3bcf50269495d..874bc49e99f59 100644 --- a/expression/builtin_miscellaneous_vec.go +++ b/expression/builtin_miscellaneous_vec.go @@ -20,9 +20,9 @@ import ( "math" "net" "strings" + "time" "github.com/google/uuid" - "github.com/pingcap/errors" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" ) @@ -281,11 +281,57 @@ func (b *builtinNameConstTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk. } func (b *builtinSleepSig) vectorized() bool { - return false + return true } +// evalInt evals a builtinSleepSig. +// See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_sleep func (b *builtinSleepSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETReal, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalReal(b.ctx, input, buf); err != nil { + return err + } + + result.ResizeInt64(n, false) + result.MergeNulls(buf) + i64s := result.Int64s() + + y := buf.Float64s() + sessVars := b.ctx.GetSessionVars() + for i := 0; i < n; i++ { + if result.IsNull(i) { + if sessVars.StrictSQLMode { + return errIncorrectArgs.GenWithStackByArgs("sleep") + } + continue + } + + if y[i] < 0 { + if sessVars.StrictSQLMode { + return errIncorrectArgs.GenWithStackByArgs("sleep") + } + i64s[i] = 0 + continue + } + + if y[i] > math.MaxFloat64/float64(time.Second.Nanoseconds()) { + return errIncorrectArgs.GenWithStackByArgs("sleep") + } + + i64s[i] = 0 + + flag := sleepDuration(sessVars, y[i]) + if flag { + i64s[i] = 1 + return nil + } + } + return nil } func (b *builtinIsIPv4MappedSig) vectorized() bool { diff --git a/expression/builtin_miscellaneous_vec_test.go b/expression/builtin_miscellaneous_vec_test.go index 9de7761cb388a..5b63e6bcbec07 100644 --- a/expression/builtin_miscellaneous_vec_test.go +++ b/expression/builtin_miscellaneous_vec_test.go @@ -14,6 +14,7 @@ package expression import ( + "math/rand" "testing" . "github.com/pingcap/check" @@ -21,6 +22,20 @@ import ( "github.com/pingcap/tidb/types" ) +type sleepTimeGener struct { + nullRation float64 +} + +func (g *sleepTimeGener) gen() interface{} { + if rand.Float64() < g.nullRation { + return nil + } + if rand.Float64() < 0.5 { + return 0 + } + return 0.1 +} + var vecBuiltinMiscellaneousCases = map[string][]vecExprBenchCase{ ast.Inet6Aton: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&ipv6StrGener{}}}, @@ -28,8 +43,15 @@ var vecBuiltinMiscellaneousCases = map[string][]vecExprBenchCase{ ast.IsIPv6: { {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}}, }, - ast.Sleep: {}, - ast.UUID: {}, + ast.Sleep: { + { + retEvalType: types.ETInt, + childrenTypes: []types.EvalType{types.ETReal}, + geners: []dataGenerator{&sleepTimeGener{0.2}}, + chunkSize: 1, + }, + }, + ast.UUID: {}, ast.Inet6Ntoa: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{ &selectStringGener{ diff --git a/expression/builtin_vectorized_test.go b/expression/builtin_vectorized_test.go index 552fbce052a5d..9d11d07b64ede 100644 --- a/expression/builtin_vectorized_test.go +++ b/expression/builtin_vectorized_test.go @@ -17,11 +17,13 @@ import ( "fmt" "math/rand" "sync" + "sync/atomic" "testing" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" @@ -783,6 +785,62 @@ func (s *testEvaluatorSuite) TestFloat32ColVec(c *C) { c.Assert(col.VecEvalReal(ctx, chk, result), IsNil) } +func (s *testEvaluatorSuite) TestSleepVec(c *C) { + ctx := mock.NewContext() + sessVars := ctx.GetSessionVars() + + fc := funcs[ast.Sleep] + // non-strict model + sessVars.StrictSQLMode = false + d := make([]types.Datum, 1) + f, err := fc.getFunction(ctx, s.datumsToConstants(d)) + c.Assert(err, IsNil) + + a := float64(3) + da := types.Datum{} + da.SetValue(a) + + tp := new(types.FieldType) + types.DefaultTypeForValue(a, tp) + input := chunk.New([]*types.FieldType{tp}, 1, 1) + buf := chunk.NewColumn(types.NewFieldType(mysql.TypeLonglong), 1) + input.AppendDatum(0, &da) + + start := time.Now() + go func() { + time.Sleep(1 * time.Second) + atomic.CompareAndSwapUint32(&ctx.GetSessionVars().Killed, 0, 1) + }() + c.Assert(f.vecEvalInt(input, buf), IsNil) + + sub := time.Since(start) + c.Assert(buf.IsNull(0), IsTrue) + c.Assert(buf.GetInt64(0), Equals, int64(1)) + c.Assert(sub.Nanoseconds(), LessEqual, int64(2*1e9)) + c.Assert(sub.Nanoseconds(), GreaterEqual, int64(1*1e9)) + + n := 3 + input = chunk.New([]*types.FieldType{tp}, n, n) + buf = chunk.NewColumn(types.NewFieldType(mysql.TypeLonglong), n) + for i := 0; i < n; i++ { + input.AppendDatum(0, &da) + } + start = time.Now() + go func() { + time.Sleep(3 * time.Second) + atomic.CompareAndSwapUint32(&ctx.GetSessionVars().Killed, 0, 1) + }() + c.Assert(f.vecEvalInt(input, buf), IsNil) + + sub = time.Since(start) + for i := 0; i < n; i++ { + c.Assert(buf.IsNull(i), IsFalse) + c.Assert(buf.GetInt64(i), Equals, int64(1)) + } + c.Assert(sub.Nanoseconds(), LessEqual, int64(4*1e9)) + c.Assert(sub.Nanoseconds(), GreaterEqual, int64(3*1e9)) +} + func BenchmarkFloat32ColRow(b *testing.B) { col, chk, _ := genFloat32Col() ctx := mock.NewContext()