Skip to content

Commit

Permalink
Merge pull request #1 from pingcap/master
Browse files Browse the repository at this point in the history
merge
  • Loading branch information
Baytwt authored Nov 11, 2019
2 parents ea01910 + e9f1997 commit e187597
Show file tree
Hide file tree
Showing 33 changed files with 678 additions and 58 deletions.
4 changes: 1 addition & 3 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -602,9 +602,7 @@ drop table if exists t;
create table t(a int);
explain select * from t where _tidb_rowid = 0;
id count task operator info
Projection_4 8000.00 root Column#1
└─TableReader_6 10000.00 root data:TableScan_5
└─TableScan_5 10000.00 cop[tikv] table:t, range:[0,0], keep order:false, stats:pseudo
Point_Get_1 1.00 root table:t, handle:0
explain select * from t where _tidb_rowid > 0;
id count task operator info
Projection_4 8000.00 root Column#1
Expand Down
11 changes: 11 additions & 0 deletions executor/point_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -564,3 +564,14 @@ func (s *testPointGetSuite) TestForUpdateRetry(c *C) {
_, err := tk.Exec("commit")
c.Assert(session.ErrForUpdateCantRetry.Equal(err), IsTrue)
}

func (s *testPointGetSuite) TestPointGetByRowID(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a varchar(20), b int)")
tk.MustExec("insert into t values(\"aaa\", 12)")
tk.MustQuery("explain select * from t where t._tidb_rowid = 1").Check(testkit.Rows(
"Point_Get_1 1.00 root table:t, handle:1"))
tk.MustQuery("select * from t where t._tidb_rowid = 1").Check(testkit.Rows("aaa 12"))
}
43 changes: 40 additions & 3 deletions expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,12 @@ func BenchmarkScalarFunctionClone(b *testing.B) {
b.ReportAllocs()
}

func getRandomTime() types.MysqlTime {
return types.FromDate(rand.Intn(2200), rand.Intn(10)+1, rand.Intn(20)+1,
rand.Intn(12), rand.Intn(60), rand.Intn(60), rand.Intn(1000000))

}

// dataGenerator is used to generate data for test.
type dataGenerator interface {
gen() interface{}
Expand All @@ -221,9 +227,9 @@ func (g *defaultGener) gen() interface{} {
return rand.Int63()
case types.ETReal:
if rand.Float64() < 0.5 {
return -rand.Float64()
return -rand.Float64() * 1000000
}
return rand.Float64()
return rand.Float64() * 1000000
case types.ETDecimal:
d := new(types.MyDecimal)
var f float64
Expand All @@ -237,7 +243,7 @@ func (g *defaultGener) gen() interface{} {
}
return d
case types.ETDatetime, types.ETTimestamp:
gt := types.FromDate(rand.Intn(2200), rand.Intn(10)+1, rand.Intn(20)+1, rand.Intn(12), rand.Intn(60), rand.Intn(60), rand.Intn(1000000))
gt := getRandomTime()
t := types.Time{Time: gt, Type: convertETType(g.eType)}
return t
case types.ETDuration:
Expand All @@ -258,6 +264,18 @@ func (g *defaultGener) gen() interface{} {
return nil
}

// selectStringGener select one string randomly from the candidates array
type selectStringGener struct {
candidates []string
}

func (g *selectStringGener) gen() interface{} {
if len(g.candidates) == 0 {
return nil
}
return g.candidates[rand.Intn(len(g.candidates))]
}

type constJSONGener struct {
jsonStr string
}
Expand All @@ -280,6 +298,13 @@ func (g *jsonStringGener) gen() interface{} {
return j.String()
}

type jsonTimeGener struct{}

func (g *jsonTimeGener) gen() interface{} {
tm := types.Time{Time: getRandomTime(), Type: mysql.TypeDatetime, Fsp: types.DefaultFsp}
return json.CreateBinary(tm.String())
}

type rangeDurationGener struct {
nullRation float64
}
Expand Down Expand Up @@ -392,6 +417,18 @@ func (g *ipv6StrGener) gen() interface{} {
return ip.String()
}

// ipv4StrGener is used to generate ipv4 strings. For example 111.111.111.111
type ipv4StrGener struct {
}

func (g *ipv4StrGener) gen() interface{} {
var ip net.IP = make([]byte, net.IPv4len)
for i := range ip {
ip[i] = uint8(rand.Intn(256))
}
return ip.String()
}

// ipv6ByteGener is used to generate ipv6 address in 16 bytes string.
type ipv6ByteGener struct {
}
Expand Down
64 changes: 59 additions & 5 deletions expression/builtin_cast_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -412,11 +412,34 @@ func (b *builtinCastRealAsJSONSig) vecEvalJSON(input *chunk.Chunk, result *chunk
}

func (b *builtinCastJSONAsRealSig) vectorized() bool {
return false
return true
}

func (b *builtinCastJSONAsRealSig) vecEvalReal(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()
buf, err := b.bufAllocator.get(types.ETJson, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf)
if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil {
return err
}

result.ResizeFloat64(n, false)
result.MergeNulls(buf)
f64s := result.Float64s()
sc := b.ctx.GetSessionVars().StmtCtx
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}
f64s[i], err = types.ConvertJSONToFloat(sc, buf.GetJSON(i))
if err != nil {
return err
}
}
return nil
}

func (b *builtinCastJSONAsTimeSig) vectorized() bool {
Expand All @@ -440,7 +463,7 @@ func (b *builtinCastJSONAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk
stmtCtx := b.ctx.GetSessionVars().StmtCtx
fsp := int8(b.tp.Decimal)
for i := 0; i < n; i++ {
if buf.IsNull(i) {
if result.IsNull(i) {
continue
}
s, err := buf.GetJSON(i).Unquote()
Expand Down Expand Up @@ -1128,11 +1151,42 @@ func (b *builtinCastTimeAsIntSig) vecEvalInt(input *chunk.Chunk, result *chunk.C
}

func (b *builtinCastTimeAsTimeSig) vectorized() bool {
return false
return true
}

func (b *builtinCastTimeAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()
if err := b.args[0].VecEvalTime(b.ctx, input, result); err != nil {
return err
}

times := result.Times()
stmt := b.ctx.GetSessionVars().StmtCtx
fsp := int8(b.tp.Decimal)
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}
res, err := times[i].Convert(stmt, b.tp.Tp)
if err != nil {
if err = handleInvalidTimeError(b.ctx, err); err != nil {
return err
}
result.SetNull(i, true)
continue
}
tm, err := res.RoundFrac(stmt, fsp)
if err != nil {
return err
}
times[i] = tm
if b.tp.Tp == mysql.TypeDate {
// Truncate hh:mm:ss part if the type is Date.
times[i].Time = types.FromDate(tm.Time.Year(), tm.Time.Month(), tm.Time.Day(), 0, 0, 0, 0)
times[i].Type = b.tp.Tp
}
}
return nil
}

func (b *builtinCastTimeAsStringSig) vectorized() bool {
Expand Down
7 changes: 7 additions & 0 deletions expression/builtin_cast_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETInt}},
{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.ETJson}},
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETDecimal}},
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETDatetime}},
{retEvalType: types.ETDuration, childrenTypes: []types.EvalType{types.ETDatetime},
Expand Down Expand Up @@ -72,6 +73,12 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{
&dataStrGener{},
}},
{retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETDuration}},
{retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETDatetime}},
{retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETTimestamp}},
{retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETJson},
geners: []dataGenerator{
&jsonTimeGener{},
}},
},
}

Expand Down
33 changes: 31 additions & 2 deletions expression/builtin_info_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@
package expression

import (
"sort"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -103,11 +106,37 @@ func (b *builtinCurrentUserSig) vecEvalString(input *chunk.Chunk, result *chunk.
}

func (b *builtinCurrentRoleSig) vectorized() bool {
return false
return true
}

// evalString evals a builtinCurrentUserSig.
// See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_current-user
func (b *builtinCurrentRoleSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()

data := b.ctx.GetSessionVars()
if data == nil || data.ActiveRoles == nil {
return errors.Errorf("Missing session variable when eval builtin")
}

result.ReserveString(n)
if len(data.ActiveRoles) == 0 {
for i := 0; i < n; i++ {
result.AppendString("")
}
return nil
}

sortedRes := make([]string, 0, 10)
for _, r := range data.ActiveRoles {
sortedRes = append(sortedRes, r.String())
}
sort.Strings(sortedRes)
res := strings.Join(sortedRes, ",")
for i := 0; i < n; i++ {
result.AppendString(res)
}
return nil
}

func (b *builtinUserSig) vectorized() bool {
Expand Down
4 changes: 3 additions & 1 deletion expression/builtin_info_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,9 @@ var vecBuiltinInfoCases = map[string][]vecExprBenchCase{
ast.RowCount: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{}},
},
ast.CurrentRole: {},
ast.CurrentRole: {
{retEvalType: types.ETString, childrenTypes: []types.EvalType{}},
},
ast.TiDBIsDDLOwner: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{}},
},
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_math_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ var vecBuiltinMathCases = map[string][]vecExprBenchCase{
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}},
},
ast.Exp: {
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}},
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}, geners: []dataGenerator{&rangeRealGener{-1, 1, 0.2}}},
},
ast.Degrees: {
{retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal}},
Expand Down
69 changes: 67 additions & 2 deletions expression/builtin_miscellaneous_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -411,11 +411,76 @@ func (b *builtinTimeAnyValueSig) vecEvalTime(input *chunk.Chunk, result *chunk.C
}

func (b *builtinInetAtonSig) vectorized() bool {
return false
return true
}

func (b *builtinInetAtonSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()
buf, err := b.bufAllocator.get(types.ETString, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf)
if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil {
return err
}
var (
byteResult, res uint64
dotCount int
)
result.ResizeInt64(n, false)
i64s := result.Int64s()
result.MergeNulls(buf)
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}
ipAddr := buf.GetString(i)
if len(ipAddr) == 0 || ipAddr[len(ipAddr)-1] == '.' {
// ip address should not end with '.'.
result.SetNull(i, true)
continue
}
//reset
byteResult = 0
res = 0
dotCount = 0
for _, c := range ipAddr {
if c >= '0' && c <= '9' {
digit := uint64(c - '0')
byteResult = byteResult*10 + digit
if byteResult > 255 {
result.SetNull(i, true)
break
}
} else if c == '.' {
dotCount++
if dotCount > 3 {
result.SetNull(i, true)
break
}
res = (res << 8) + byteResult
byteResult = 0
} else {
result.SetNull(i, true)
break // illegal char (not number or .)
}
}
// 127 -> 0.0.0.127
// 127.255 -> 127.0.0.255
// 127.256 -> NULL
// 127.2.1 -> 127.2.0.1
if !result.IsNull(i) {
if dotCount == 1 {
res <<= 16
}
if dotCount == 2 {
res <<= 8
}
i64s[i] = int64((res << 8) + byteResult)
}
}
return nil
}

func (b *builtinInet6NtoaSig) vectorized() bool {
Expand Down
17 changes: 16 additions & 1 deletion expression/builtin_miscellaneous_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,22 @@ var vecBuiltinMiscellaneousCases = map[string][]vecExprBenchCase{
ast.Sleep: {},
ast.UUID: {},
ast.Inet6Ntoa: {},
ast.InetAton: {},
ast.InetAton: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&ipv4StrGener{}}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{
&selectStringGener{
candidates: []string{
"11.11.11.11.", // last char is .
"266.266.266.266", // int in string exceed 255
"127",
".122",
".123.123",
"127.255",
"127.2.1",
},
}}},
},
ast.IsIPv4Mapped: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&ipv4MappedByteGener{}}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&ipv6ByteGener{}}},
Expand Down
Loading

0 comments on commit e187597

Please sign in to comment.