Skip to content

Commit

Permalink
Merge branch 'master' into rollback-reason
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao committed Nov 18, 2019
2 parents fe11fb9 + 66d99a1 commit 2ff5e70
Show file tree
Hide file tree
Showing 28 changed files with 774 additions and 100 deletions.
10 changes: 7 additions & 3 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1029,9 +1029,13 @@ LOOP:
return
}

// test index range
for i := 0; i < 100; i++ {
index := rand.Intn(len(keys) - 3)
// Test index range with lower/upper boundary and random inner cases
step := len(keys) / 20
for i := 0; i <= 20; i++ {
index := i * step
if index > len(keys)-3 {
index = len(keys) - 3
}
rows := tk.MustQuery("select c1 from test_add_index where c3 >= ? order by c1 limit 3", keys[index]).Rows()
matchRows(c, rows, [][]interface{}{{keys[index]}, {keys[index+1]}, {keys[index+2]}})
}
Expand Down
15 changes: 8 additions & 7 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1847,13 +1847,14 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex
compareFuncs: v.OuterCompareFuncs,
},
innerMergeCtx: innerMergeCtx{
readerBuilder: &dataReaderBuilder{Plan: innerPlan, executorBuilder: b},
rowTypes: innerTypes,
joinKeys: v.InnerJoinKeys,
keyCols: innerKeyCols,
compareFuncs: v.CompareFuncs,
colLens: v.IdxColLens,
desc: v.Desc,
readerBuilder: &dataReaderBuilder{Plan: innerPlan, executorBuilder: b},
rowTypes: innerTypes,
joinKeys: v.InnerJoinKeys,
keyCols: innerKeyCols,
compareFuncs: v.CompareFuncs,
colLens: v.IdxColLens,
desc: v.Desc,
keyOff2KeyOffOrderByIdx: v.KeyOff2KeyOffOrderByIdx,
},
workerWg: new(sync.WaitGroup),
isOuterJoin: v.JoinType.IsOuterJoin(),
Expand Down
36 changes: 21 additions & 15 deletions executor/index_lookup_merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,13 +76,14 @@ type outerMergeCtx struct {
}

type innerMergeCtx struct {
readerBuilder *dataReaderBuilder
rowTypes []*types.FieldType
joinKeys []*expression.Column
keyCols []int
compareFuncs []expression.CompareFunc
colLens []int
desc bool
readerBuilder *dataReaderBuilder
rowTypes []*types.FieldType
joinKeys []*expression.Column
keyCols []int
compareFuncs []expression.CompareFunc
colLens []int
desc bool
keyOff2KeyOffOrderByIdx []int
}

type lookUpMergeJoinTask struct {
Expand Down Expand Up @@ -423,15 +424,20 @@ func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJo
sort.Slice(task.outerOrderIdx, func(i, j int) bool {
idxI, idxJ := task.outerOrderIdx[i], task.outerOrderIdx[j]
rowI, rowJ := task.outerResult.GetRow(idxI), task.outerResult.GetRow(idxJ)
for id, joinKey := range imw.outerMergeCtx.joinKeys {
cmp, _, err := imw.outerMergeCtx.compareFuncs[id](imw.ctx, joinKey, joinKey, rowI, rowJ)
var cmp int64
var err error
for _, keyOff := range imw.keyOff2KeyOffOrderByIdx {
joinKey := imw.outerMergeCtx.joinKeys[keyOff]
cmp, _, err = imw.outerMergeCtx.compareFuncs[keyOff](imw.ctx, joinKey, joinKey, rowI, rowJ)
terror.Log(err)
if cmp != 0 || imw.nextColCompareFilters == nil {
return cmp < 0
if cmp != 0 {
break
}
return imw.nextColCompareFilters.CompareRow(rowI, rowJ) < 0
}
return false
if cmp != 0 || imw.nextColCompareFilters == nil {
return cmp < 0
}
return imw.nextColCompareFilters.CompareRow(rowI, rowJ) < 0
})
}
dLookUpKeys, err := imw.constructDatumLookupKeys(task)
Expand Down Expand Up @@ -571,8 +577,8 @@ func (imw *innerMergeWorker) fetchInnerRowsWithSameKey(ctx context.Context, task
}

func (imw *innerMergeWorker) compare(outerRow, innerRow chunk.Row) (int, error) {
for i := 0; i < len(imw.outerMergeCtx.joinKeys); i++ {
cmp, _, err := imw.innerMergeCtx.compareFuncs[i](imw.ctx, imw.outerMergeCtx.joinKeys[i], imw.innerMergeCtx.joinKeys[i], outerRow, innerRow)
for _, keyOff := range imw.innerMergeCtx.keyOff2KeyOffOrderByIdx {
cmp, _, err := imw.innerMergeCtx.compareFuncs[keyOff](imw.ctx, imw.outerMergeCtx.joinKeys[keyOff], imw.innerMergeCtx.joinKeys[keyOff], outerRow, innerRow)
if err != nil || cmp != 0 {
return int(cmp), err
}
Expand Down
20 changes: 20 additions & 0 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1589,3 +1589,23 @@ func (s *testSuiteJoin1) TestIssue11390(c *C) {
tk.MustExec("insert into 11390t values(1, 1)")
tk.MustQuery("select /*+ TIDB_INLJ(t1, t2) */ * from 11390t t1, 11390t t2 where t1.k2 > 0 and t1.k2 = t2.k2 and t2.k1=1;").Check(testkit.Rows("1 1 1 1"))
}

func (s *testSuiteJoin1) TestIssue13177(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1(a varchar(20), b int, c int)")
tk.MustExec("create table t2(a varchar(20), b int, c int, primary key(a, b))")
tk.MustExec("insert into t1 values(\"abcd\", 1, 1), (\"bacd\", 2, 2), (\"cbad\", 3, 3)")
tk.MustExec("insert into t2 values(\"bcd\", 1, 1), (\"acd\", 2, 2), (\"bad\", 3, 3)")
tk.MustQuery("select /*+ inl_merge_join(t1, t2) */ * from t1 join t2 on substr(t1.a, 2, 4) = t2.a and t1.b = t2.b where t1.c between 1 and 5").Check(testkit.Rows(
"bacd 2 2 acd 2 2",
"cbad 3 3 bad 3 3",
"abcd 1 1 bcd 1 1",
))
tk.MustQuery("select /*+ inl_merge_join(t1, t2) */ t1.* from t1 join t2 on substr(t1.a, 2, 4) = t2.a and t1.b = t2.b where t1.c between 1 and 5").Check(testkit.Rows(
"bacd 2 2",
"cbad 3 3",
"abcd 1 1",
))
}
53 changes: 41 additions & 12 deletions expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
Expand Down Expand Up @@ -692,6 +693,8 @@ type vecExprBenchCase struct {
// geners[gen1, gen2] will be regarded as geners[gen1, gen2, nil].
// This field is optional.
geners []dataGenerator
// constants are used to generate constant data for children[i].
constants []*Constant
}

type vecExprBenchCases map[string][]vecExprBenchCase
Expand All @@ -711,7 +714,7 @@ func fillColumnWithGener(eType types.EvalType, chk *chunk.Chunk, colIdx int, gen
}

col := chk.Column(colIdx)
col.Reset()
col.Reset(eType)
for i := 0; i < batchSize; i++ {
v := gen.gen()
if v == nil {
Expand Down Expand Up @@ -787,7 +790,11 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx
input = chunk.New(fts, 1024, 1024)
for i, eType := range testCase.childrenTypes {
fillColumn(eType, input, i, testCase)
cols[i] = &Column{Index: i, RetType: fts[i]}
if i < len(testCase.constants) && testCase.constants[i] != nil {
cols[i] = testCase.constants[i]
} else {
cols[i] = &Column{Index: i, RetType: fts[i]}
}
}

expr, err := NewFunction(ctx, funcName, eType2FieldType(testCase.retEvalType), cols...)
Expand Down Expand Up @@ -919,7 +926,11 @@ func genVecBuiltinFuncBenchCase(ctx sessionctx.Context, funcName string, testCas
input = chunk.New(fts, 1024, 1024)
for i, eType := range testCase.childrenTypes {
fillColumn(eType, input, i, testCase)
cols[i] = &Column{Index: i, RetType: fts[i]}
if i < len(testCase.constants) && testCase.constants[i] != nil {
cols[i] = testCase.constants[i]
} else {
cols[i] = &Column{Index: i, RetType: fts[i]}
}
}
if len(cols) == 0 {
input.SetNumVirtualRows(1024)
Expand Down Expand Up @@ -991,10 +1002,19 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) {
for funcName, testCases := range vecExprCases {
for _, testCase := range testCases {
ctx := mock.NewContext()
if funcName == ast.AesEncrypt {
if funcName == ast.AesDecrypt || funcName == ast.AesEncrypt {
err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb")
c.Assert(err, IsNil)
}
if funcName == ast.CurrentUser || funcName == ast.User {
ctx.GetSessionVars().User = &auth.UserIdentity{
Username: "tidb",
Hostname: "localhost",
CurrentUser: true,
AuthHostname: "localhost",
AuthUsername: "tidb",
}
}
baseFunc, fts, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase)
baseFuncName := fmt.Sprintf("%v", reflect.TypeOf(baseFunc))
tmp := strings.Split(baseFuncName, ".")
Expand Down Expand Up @@ -1185,12 +1205,21 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
for funcName, testCases := range vecExprCases {
for _, testCase := range testCases {
if funcName == ast.AesEncrypt {
if funcName == ast.AesDecrypt || funcName == ast.AesEncrypt {
err := ctx.GetSessionVars().SetSystemVar(variable.BlockEncryptionMode, "aes-128-ecb")
if err != nil {
panic(err)
}
}
if funcName == ast.CurrentUser || funcName == ast.User {
ctx.GetSessionVars().User = &auth.UserIdentity{
Username: "tidb",
Hostname: "localhost",
CurrentUser: true,
AuthHostname: "localhost",
AuthUsername: "tidb",
}
}
baseFunc, _, input, output := genVecBuiltinFuncBenchCase(ctx, funcName, testCase)
baseFuncName := fmt.Sprintf("%v", reflect.TypeOf(baseFunc))
tmp := strings.Split(baseFuncName, ".")
Expand Down Expand Up @@ -1255,7 +1284,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
switch testCase.retEvalType {
case types.ETInt:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalInt(row)
if err != nil {
Expand All @@ -1270,7 +1299,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETReal:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalReal(row)
if err != nil {
Expand All @@ -1285,7 +1314,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETDecimal:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalDecimal(row)
if err != nil {
Expand All @@ -1300,7 +1329,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETDatetime, types.ETTimestamp:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalTime(row)
if err != nil {
Expand All @@ -1315,7 +1344,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETDuration:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalDuration(row)
if err != nil {
Expand All @@ -1330,7 +1359,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETJson:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalJSON(row)
if err != nil {
Expand All @@ -1345,7 +1374,7 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases
}
case types.ETString:
for i := 0; i < b.N; i++ {
output.Reset()
output.Reset(testCase.retEvalType)
for row := it.Begin(); row != it.End(); row = it.Next() {
v, isNull, err := baseFunc.evalString(row)
if err != nil {
Expand Down
39 changes: 37 additions & 2 deletions expression/builtin_compare_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,11 +164,35 @@ func (b *builtinGreatestIntSig) vectorized() bool {
}

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

func (b *builtinGEIntSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
return errors.Errorf("not implemented")
n := input.NumRows()
var err error
var buf0, buf1 *chunk.Column
buf0, err = b.bufAllocator.get(types.ETInt, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf0)
if err = b.args[0].VecEvalInt(b.ctx, input, buf0); err != nil {
return err
}
buf1, err = b.bufAllocator.get(types.ETInt, n)
if err != nil {
return err
}
defer b.bufAllocator.put(buf1)
if err = b.args[1].VecEvalInt(b.ctx, input, buf1); err != nil {
return err
}

result.ResizeInt64(n, false)
vecCompareInt(mysql.HasUnsignedFlag(b.args[0].GetType().Flag), mysql.HasUnsignedFlag(b.args[1].GetType().Flag), buf0, buf1, result)
result.MergeNulls(buf0, buf1)
vecResOfGE(result.Int64s())
return nil
}

func (b *builtinLeastRealSig) vectorized() bool {
Expand Down Expand Up @@ -540,6 +564,17 @@ func vecResOfGT(res []int64) {
}
}

func vecResOfGE(res []int64) {
n := len(res)
for i := 0; i < n; i++ {
if res[i] >= 0 {
res[i] = 1
} else {
res[i] = 0
}
}
}

//vecCompareInt is vectorized CompareInt()
func vecCompareInt(isUnsigned0, isUnsigned1 bool, largs, rargs, result *chunk.Column) {
switch {
Expand Down
21 changes: 19 additions & 2 deletions expression/builtin_compare_vec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,8 +98,25 @@ var vecBuiltinCompareCases = map[string][]vecExprBenchCase{
},
},
},
ast.EQ: {},
ast.GE: {},
ast.EQ: {},
ast.GE: {
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt},
childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag},
{Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag},
},
},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt},
childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeLonglong},
{Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag},
},
},
{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt},
childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag},
{Tp: mysql.TypeLonglong},
},
},
},
ast.Date: {},
ast.Greatest: {
{retEvalType: types.ETDecimal, childrenTypes: []types.EvalType{types.ETDecimal, types.ETDecimal, types.ETDecimal}},
Expand Down
Loading

0 comments on commit 2ff5e70

Please sign in to comment.