Skip to content

Commit

Permalink
Merge branch 'master' into droprole
Browse files Browse the repository at this point in the history
  • Loading branch information
Lingyu Song authored Mar 13, 2019
2 parents bbceb18 + f5a4dd9 commit b49fe4d
Show file tree
Hide file tree
Showing 18 changed files with 240 additions and 69 deletions.
12 changes: 11 additions & 1 deletion CONTRIBUTING.md
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ TiDB is written in [Go](http://golang.org).
If you don't have a Go development environment,
please [set one up](http://golang.org/doc/code.html).

The version of GO should be **1.11** or above.
The version of GO should be **1.12** or above.

After installation, there are two ways to build TiDB binary.

Expand Down Expand Up @@ -215,6 +215,8 @@ mysql -h127.0.0.1 -P4000 -uroot test --default-character-set utf8

#### Run Test

Run all tests

```sh
# Run unit test to make sure all test passed.
make dev
Expand All @@ -223,6 +225,14 @@ make dev
make checklist
```

You can also run a single test in a file. For example, if you want to run
test `TestToInt64` in file `types/datum.go`, you can do something like

```sh
cd types
GO111MODULE=on go test -check.f TestToInt64
```

### Step 5: Keep your branch in sync

```sh
Expand Down
9 changes: 9 additions & 0 deletions executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,8 @@ func BuildWindowFunctions(ctx sessionctx.Context, windowFuncDesc *aggregation.Ag
return buildCumeDist(ordinal, orderByCols)
case ast.WindowFuncNthValue:
return buildNthValue(windowFuncDesc, ordinal)
case ast.WindowFuncPercentRank:
return buildPercenRank(ordinal, orderByCols)
default:
return Build(ctx, windowFuncDesc, ordinal)
}
Expand Down Expand Up @@ -386,3 +388,10 @@ func buildNthValue(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc {
nth, _, _ := expression.GetUint64FromConstant(aggFuncDesc.Args[1])
return &nthValue{baseAggFunc: base, tp: aggFuncDesc.RetTp, nth: nth}
}

func buildPercenRank(ordinal int, orderByCols []*expression.Column) AggFunc {
base := baseAggFunc{
ordinal: ordinal,
}
return &percentRank{baseAggFunc: base, rowComparer: buildRowComparer(orderByCols)}
}
61 changes: 61 additions & 0 deletions executor/aggfuncs/func_percent_rank.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
// 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 aggfuncs

import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
)

// percentRank calculates the percentage of partition values less than the value in the current row, excluding the highest value.
// It can be calculated as `(rank - 1) / (total_rows_in_set - 1).
type percentRank struct {
baseAggFunc
rowComparer
}

func (pr *percentRank) AllocPartialResult() PartialResult {
return PartialResult(&partialResult4Rank{})
}

func (pr *percentRank) ResetPartialResult(partial PartialResult) {
p := (*partialResult4Rank)(partial)
p.curIdx = 0
p.lastRank = 0
p.rows = p.rows[:0]
}

func (pr *percentRank) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, partial PartialResult) error {
p := (*partialResult4Rank)(partial)
p.rows = append(p.rows, rowsInGroup...)
return nil
}

func (pr *percentRank) AppendFinalResult2Chunk(sctx sessionctx.Context, partial PartialResult, chk *chunk.Chunk) error {
p := (*partialResult4Rank)(partial)
numRows := int64(len(p.rows))
p.curIdx++
if p.curIdx == 1 {
p.lastRank = 1
chk.AppendFloat64(pr.ordinal, 0)
return nil
}
if pr.compareRows(p.rows[p.curIdx-2], p.rows[p.curIdx-1]) == 0 {
chk.AppendFloat64(pr.ordinal, float64(p.lastRank-1)/float64(numRows-1))
return nil
}
p.lastRank = p.curIdx
chk.AppendFloat64(pr.ordinal, float64(p.lastRank-1)/float64(numRows-1))
return nil
}
15 changes: 13 additions & 2 deletions executor/chunk_size_control_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"context"
"fmt"
"strings"
"sync"
"time"

. "github.com/pingcap/check"
Expand All @@ -39,22 +40,32 @@ var (
)

type testSlowClient struct {
sync.RWMutex
tikv.Client
regionDelay map[uint64]time.Duration
}

func (c *testSlowClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
regionID := req.RegionId
if req.Type == tikvrpc.CmdCop && c.regionDelay[regionID] > 0 {
time.Sleep(c.regionDelay[regionID])
delay := c.GetDelay(regionID)
if req.Type == tikvrpc.CmdCop && delay > 0 {
time.Sleep(delay)
}
return c.Client.SendRequest(ctx, addr, req, timeout)
}

func (c *testSlowClient) SetDelay(regionID uint64, dur time.Duration) {
c.Lock()
defer c.Unlock()
c.regionDelay[regionID] = dur
}

func (c *testSlowClient) GetDelay(regionID uint64) time.Duration {
c.RLock()
defer c.RUnlock()
return c.regionDelay[regionID]
}

// manipulateCluster splits this cluster's region by splitKeys and returns regionIDs after split
func manipulateCluster(cluster *mocktikv.Cluster, splitKeys [][]byte) []uint64 {
regions := cluster.GetAllRegions()
Expand Down
9 changes: 4 additions & 5 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1287,11 +1287,10 @@ func (e *UnionExec) Close() error {
// Before every execution, we must clear statement context.
func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
vars := ctx.GetSessionVars()
sc := new(stmtctx.StatementContext)
sc.TimeZone = vars.Location()
sc.MemTracker = memory.NewTracker(s.Text(), vars.MemQuotaQuery)
sc.NowTs = time.Time{}
sc.SysTs = time.Time{}
sc := &stmtctx.StatementContext{
TimeZone: vars.Location(),
MemTracker: memory.NewTracker(s.Text(), vars.MemQuotaQuery),
}
switch config.GetGlobalConfig().OOMAction {
case config.OOMActionCancel:
sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{})
Expand Down
7 changes: 7 additions & 0 deletions executor/window_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,4 +119,11 @@ func (s *testSuite2) TestWindowFunctions(c *C) {
result.Check(testkit.Rows("1 2", "1 2", "2 2", "2 2"))
result = tk.MustQuery("select a, nth_value(a, 5) over() from t")
result.Check(testkit.Rows("1 <nil>", "1 <nil>", "2 <nil>", "2 <nil>"))

result = tk.MustQuery("select a, percent_rank() over() from t")
result.Check(testkit.Rows("1 0", "1 0", "2 0", "2 0"))
result = tk.MustQuery("select a, percent_rank() over(order by a) from t")
result.Check(testkit.Rows("1 0", "1 0", "2 0.6666666666666666", "2 0.6666666666666666"))
result = tk.MustQuery("select a, b, percent_rank() over(order by a, b) from t")
result.Check(testkit.Rows("1 1 0", "1 2 0.3333333333333333", "2 1 0.6666666666666666", "2 2 1"))
}
7 changes: 7 additions & 0 deletions expression/aggregation/base_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,8 @@ func (a *baseFuncDesc) typeInfer(ctx sessionctx.Context) {
a.typeInfer4NumberFuncs()
case ast.WindowFuncCumeDist:
a.typeInfer4CumeDist()
case ast.WindowFuncPercentRank:
a.typeInfer4PercentRank()
default:
panic("unsupported agg function: " + a.Name)
}
Expand Down Expand Up @@ -200,6 +202,11 @@ func (a *baseFuncDesc) typeInfer4CumeDist() {
a.RetTp.Flen, a.RetTp.Decimal = mysql.MaxRealWidth, mysql.NotFixedDec
}

func (a *baseFuncDesc) typeInfer4PercentRank() {
a.RetTp = types.NewFieldType(mysql.TypeDouble)
a.RetTp.Flag, a.RetTp.Decimal = mysql.MaxRealWidth, mysql.NotFixedDec
}

// GetDefaultValue gets the default value when the function's input is null.
// According to MySQL, default values of the function are listed as follows:
// e.g.
Expand Down
10 changes: 5 additions & 5 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -465,7 +465,7 @@ func (b *builtinCastIntAsRealSig) evalReal(row chunk.Row) (res float64, isNull b
} else {
var uVal uint64
sc := b.ctx.GetSessionVars().StmtCtx
uVal, err = types.ConvertIntToUint(sc, val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
uVal, err = types.ConvertIntToUint(sc, val, types.IntergerUnsignedUpperBound(mysql.TypeLonglong), mysql.TypeLonglong)
res = float64(uVal)
}
return res, false, err
Expand Down Expand Up @@ -493,7 +493,7 @@ func (b *builtinCastIntAsDecimalSig) evalDecimal(row chunk.Row) (res *types.MyDe
} else {
var uVal uint64
sc := b.ctx.GetSessionVars().StmtCtx
uVal, err = types.ConvertIntToUint(sc, val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
uVal, err = types.ConvertIntToUint(sc, val, types.IntergerUnsignedUpperBound(mysql.TypeLonglong), mysql.TypeLonglong)
if err != nil {
return res, false, err
}
Expand Down Expand Up @@ -523,7 +523,7 @@ func (b *builtinCastIntAsStringSig) evalString(row chunk.Row) (res string, isNul
} else {
var uVal uint64
sc := b.ctx.GetSessionVars().StmtCtx
uVal, err = types.ConvertIntToUint(sc, val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeLonglong)
uVal, err = types.ConvertIntToUint(sc, val, types.IntergerUnsignedUpperBound(mysql.TypeLonglong), mysql.TypeLonglong)
if err != nil {
return res, false, err
}
Expand Down Expand Up @@ -748,13 +748,13 @@ func (b *builtinCastRealAsIntSig) evalInt(row chunk.Row) (res int64, isNull bool
return res, isNull, err
}
if !mysql.HasUnsignedFlag(b.tp.Flag) {
res, err = types.ConvertFloatToInt(val, types.SignedLowerBound[mysql.TypeLonglong], types.SignedUpperBound[mysql.TypeLonglong], mysql.TypeDouble)
res, err = types.ConvertFloatToInt(val, types.IntergerSignedLowerBound(mysql.TypeLonglong), types.IntergerSignedUpperBound(mysql.TypeLonglong), mysql.TypeDouble)
} else if b.inUnion && val < 0 {
res = 0
} else {
var uintVal uint64
sc := b.ctx.GetSessionVars().StmtCtx
uintVal, err = types.ConvertFloatToUint(sc, val, types.UnsignedUpperBound[mysql.TypeLonglong], mysql.TypeDouble)
uintVal, err = types.ConvertFloatToUint(sc, val, types.IntergerUnsignedUpperBound(mysql.TypeLonglong), mysql.TypeDouble)
res = int64(uintVal)
}
return res, isNull, err
Expand Down
37 changes: 37 additions & 0 deletions planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -887,3 +887,40 @@ func BenchmarkOptimize(b *testing.B) {
})
}
}

func (s *testAnalyzeSuite) TestIssue9562(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()

tk.MustExec("use test")
tk.MustExec("create table t1(a bigint, b bigint, c bigint)")
tk.MustExec("create table t2(a bigint, b bigint, c bigint, index idx(a, b, c))")

tk.MustQuery("explain select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t2.a=t1.a and t2.b>t1.b-1 and t2.b<t1.b+1 and t2.c=t1.c;").Check(testkit.Rows(
"IndexJoin_9 12475.01 root inner join, inner:IndexReader_8, outer key:test.t1.a, inner key:test.t2.a, other cond:eq(test.t1.c, test.t2.c), gt(test.t2.b, minus(test.t1.b, 1)), lt(test.t2.b, plus(test.t1.b, 1))",
"├─TableReader_12 9980.01 root data:Selection_11",
"│ └─Selection_11 9980.01 cop not(isnull(test.t1.a)), not(isnull(test.t1.c))",
"│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo",
"└─IndexReader_8 0.00 root index:Selection_7",
" └─Selection_7 0.00 cop not(isnull(test.t2.a)), not(isnull(test.t2.c))",
" └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [test.t1.a test.t1.c], keep order:false, stats:pseudo",
))

tk.MustExec("create table t(a int, b int, index idx_ab(a, b))")
tk.MustQuery("explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null").Check(testkit.Rows(
"Projection_7 0.00 root t1.a, t1.b, t2.a, t2.b",
"└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)]",
" ├─TableReader_12 0.00 root data:Selection_11",
" │ └─Selection_11 0.00 cop isnull(t2.b), not(isnull(t2.b))",
" │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo",
" └─TableReader_15 9990.00 root data:Selection_14",
" └─Selection_14 9990.00 cop not(isnull(t1.b))",
" └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo",
))
}
22 changes: 18 additions & 4 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -562,14 +562,28 @@ func (p *LogicalJoin) buildRangeForIndexJoin(indexInfo *model.IndexInfo, innerPl
return nil, nil, nil
}

// We should guarantee that all the join's equal condition is used.
for _, eqCond := range eqConds {
// Guarantee res.AccessConds is not empty.
if len(res.AccessConds) == 0 {
return nil, nil, nil
}

// Find invalid fake condition and modify the joinKey's idxOff to -1.
var invalidFakeConds []expression.Expression
for i, eqCond := range eqConds {
if !expression.Contains(res.AccessConds, eqCond) {
return nil, nil, nil
keyOff2IdxOff[i] = -1
invalidFakeConds = append(invalidFakeConds, eqCond)
}
}

// Filter out invalidFakeConds from res.RemainedConds.
for _, cond := range res.RemainedConds {
if !expression.Contains(invalidFakeConds, cond) {
remained = append(remained, cond)
}
}

return res.Ranges, append(remained, res.RemainedConds...), keyOff2IdxOff
return res.Ranges, remained, keyOff2IdxOff
}

func (p *LogicalJoin) buildFakeEqCondsForIndexJoin(keys, idxCols []*expression.Column, colLengths []int,
Expand Down
2 changes: 1 addition & 1 deletion server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -585,7 +585,7 @@ func (cc *clientConn) Run(ctx context.Context) {
}

startTime := time.Now()
if err = cc.dispatch(logutil.WithRecvTs(ctx, startTime.UnixNano()/int64(time.Millisecond)), data); err != nil {
if err = cc.dispatch(ctx, data); err != nil {
if terror.ErrorEqual(err, io.EOF) {
cc.addMetrics(data[0], startTime, nil)
return
Expand Down
6 changes: 3 additions & 3 deletions statistics/feedback.go
Original file line number Diff line number Diff line change
Expand Up @@ -1152,9 +1152,9 @@ func getMaxValue(ft *types.FieldType) (max types.Datum) {
switch ft.Tp {
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong:
if mysql.HasUnsignedFlag(ft.Flag) {
max.SetUint64(types.UnsignedUpperBound[ft.Tp])
max.SetUint64(types.IntergerUnsignedUpperBound(ft.Tp))
} else {
max.SetInt64(types.SignedUpperBound[ft.Tp])
max.SetInt64(types.IntergerSignedUpperBound(ft.Tp))
}
case mysql.TypeFloat:
max.SetFloat32(float32(types.GetMaxFloat(ft.Flen, ft.Decimal)))
Expand Down Expand Up @@ -1188,7 +1188,7 @@ func getMinValue(ft *types.FieldType) (min types.Datum) {
if mysql.HasUnsignedFlag(ft.Flag) {
min.SetUint64(0)
} else {
min.SetInt64(types.SignedLowerBound[ft.Tp])
min.SetInt64(types.IntergerSignedLowerBound(ft.Tp))
}
case mysql.TypeFloat:
min.SetFloat32(float32(-types.GetMaxFloat(ft.Flen, ft.Decimal)))
Expand Down
4 changes: 2 additions & 2 deletions statistics/histogram.go
Original file line number Diff line number Diff line change
Expand Up @@ -894,7 +894,7 @@ type countByRangeFunc = func(*stmtctx.StatementContext, int64, []*ranger.Range)
// Also, there're redundant calculation with Selectivity(). We need to reduce it too.
func newHistogramBySelectivity(sc *stmtctx.StatementContext, histID int64, oldHist, newHist *Histogram, ranges []*ranger.Range, cntByRangeFunc countByRangeFunc) error {
cntPerVal := int64(oldHist.AvgCountPerNotNullValue(int64(oldHist.totalRowCount())))
var totCnt int64 = 0
var totCnt int64
for boundIdx, ranIdx, highRangeIdx := 0, 0, 0; boundIdx < oldHist.Bounds.NumRows() && ranIdx < len(ranges); boundIdx, ranIdx = boundIdx+2, highRangeIdx {
for highRangeIdx < len(ranges) && chunk.Compare(oldHist.Bounds.GetRow(boundIdx+1), 0, &ranges[highRangeIdx].HighVal[0]) >= 0 {
highRangeIdx++
Expand Down Expand Up @@ -941,7 +941,7 @@ func (idx *Index) newIndexBySelectivity(sc *stmtctx.StatementContext, statsNode
newIndexHist.Histogram = *NewHistogram(idx.ID, int64(float64(idx.NDV)*statsNode.Selectivity), 0, 0, types.NewFieldType(mysql.TypeBlob), chunk.InitialCapacity, 0)

lowBucketIdx, highBucketIdx := 0, 0
var totCnt int64 = 0
var totCnt int64

// Bucket bound of index is encoded one, so we need to decode it if we want to calculate the fraction accurately.
// TODO: enhance its calculation.
Expand Down
2 changes: 1 addition & 1 deletion store/tikv/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ func (b *Backoffer) Backoff(typ backoffType, err error) error {
b.totalSleep += f(b.ctx)
b.types = append(b.types, typ)

var startTs interface{} = ""
var startTs interface{}
if ts := b.ctx.Value(txnStartKey); ts != nil {
startTs = ts
}
Expand Down
Loading

0 comments on commit b49fe4d

Please sign in to comment.