Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into explain…
Browse files Browse the repository at this point in the history
…_ctx
  • Loading branch information
XuHuaiyu committed Aug 13, 2019
2 parents 661eae4 + 922f569 commit c1c5c07
Show file tree
Hide file tree
Showing 378 changed files with 16,773 additions and 5,218 deletions.
281 changes: 281 additions & 0 deletions CHANGELOG.md

Large diffs are not rendered by default.

10 changes: 7 additions & 3 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,14 @@ path_to_add := $(addsuffix /bin,$(subst :,/bin:,$(GOPATH))):$(PWD)/tools/bin
export PATH := $(path_to_add):$(PATH)

GO := GO111MODULE=on go
GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG)
GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) -tags codes
GOTEST := CGO_ENABLED=1 $(GO) test -p 4
OVERALLS := CGO_ENABLED=1 GO111MODULE=on overalls

ARCH := "`uname -s`"
LINUX := "Linux"
MAC := "Darwin"
PACKAGE_LIST := go list ./...| grep -vE "cmd"
PACKAGE_LIST := go list ./...| grep -vE "cmd" | grep -vE "test"
PACKAGES := $$($(PACKAGE_LIST))
PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||'
FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go")
Expand Down Expand Up @@ -61,7 +61,7 @@ build:
# Install the check tools.
check-setup:tools/bin/revive tools/bin/goword tools/bin/gometalinter tools/bin/gosec

check: fmt errcheck lint tidy check-static vet
check: fmt errcheck lint tidy testSuite check-static vet

# These need to be fixed before they can be ran regularly
check-fail: goword check-slow
Expand Down Expand Up @@ -106,6 +106,10 @@ tidy:
@echo "go mod tidy"
./tools/check/check-tidy.sh

testSuite:
@echo "testSuite"
./tools/check/check_testSuite.sh

clean:
$(GO) clean -i ./...
rm -rf *.out
Expand Down
1 change: 1 addition & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
[![GitHub release date](https://img.shields.io/github/release-date/pingcap/tidb.svg)](https://github.com/pingcap/tidb/releases)
[![CircleCI Status](https://circleci.com/gh/pingcap/tidb.svg?style=shield)](https://circleci.com/gh/pingcap/tidb)
[![Coverage Status](https://codecov.io/gh/pingcap/tidb/branch/master/graph/badge.svg)](https://codecov.io/gh/pingcap/tidb)
[![GoDoc](https://img.shields.io/badge/Godoc-reference-blue.svg)](https://godoc.org/github.com/pingcap/tidb)

- [**Stack Overflow**](https://stackoverflow.com/questions/tagged/tidb)
- Community [**Slack Channel**](https://pingcap.com/tidbslack/)
Expand Down
38 changes: 35 additions & 3 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,21 +19,22 @@ import (
"fmt"
"os"
"testing"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/parser"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
dto "github.com/prometheus/client_model/go"
)

func TestT(t *testing.T) {
Expand Down Expand Up @@ -146,13 +147,21 @@ func (s *testSuite) TestGlobalBinding(c *C) {
tk.MustExec("create table t1(i int, s varchar(20))")
tk.MustExec("create index index_t on t(i,s)")

metrics.BindTotalGauge.Reset()
metrics.BindMemoryUsage.Reset()

_, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100")
c.Assert(err, IsNil, Commentf("err %v", err))

time.Sleep(time.Second * 1)
_, err = tk.Exec("create global binding for select * from t where i>99 using select * from t use index(index_t) where i>99")
c.Assert(err, IsNil)

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

sql, hash := parser.NormalizeDigest("select * from t where i > ?")

bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test")
Expand Down Expand Up @@ -203,6 +212,12 @@ func (s *testSuite) TestGlobalBinding(c *C) {
bindData = s.domain.BindHandle().GetBindRecord(hash, sql, "test")
c.Check(bindData, IsNil)

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
// From newly created global bind handle.
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

bindHandle = bindinfo.NewBindHandle(tk.Se)
err = bindHandle.Update(true)
c.Check(err, IsNil)
Expand Down Expand Up @@ -235,13 +250,21 @@ func (s *testSuite) TestSessionBinding(c *C) {
tk.MustExec("create table t1(i int, s varchar(20))")
tk.MustExec("create index index_t on t(i,s)")

metrics.BindTotalGauge.Reset()
metrics.BindMemoryUsage.Reset()

_, err := tk.Exec("create session binding for select * from t where i>100 using select * from t use index(index_t) where i>100")
c.Assert(err, IsNil, Commentf("err %v", err))

time.Sleep(time.Second * 1)
_, err = tk.Exec("create session binding for select * from t where i>99 using select * from t use index(index_t) where i>99")
c.Assert(err, IsNil)

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(161))

handle := tk.Se.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
bindData := handle.GetBindRecord("select * from t where i > ?", "test")
c.Check(bindData, NotNil)
Expand Down Expand Up @@ -283,6 +306,11 @@ func (s *testSuite) TestSessionBinding(c *C) {
c.Check(bindData, NotNil)
c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?")
c.Check(bindData.Status, Equals, "deleted")

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
}

func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {
Expand Down Expand Up @@ -318,6 +346,7 @@ func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {

tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id")

metrics.BindUsageCounter.Reset()
tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows(
"MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id",
"├─Sort_11 9990.00 root test.t1.id:asc",
Expand All @@ -329,6 +358,9 @@ func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {
" └─Selection_13 9990.00 cop not(isnull(test.t2.id))",
" └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo",
))
pb := &dto.Metric{}
metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Write(pb)
c.Assert(pb.GetCounter().GetValue(), Equals, float64(1))

tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id")

Expand Down
19 changes: 19 additions & 0 deletions bindinfo/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,10 @@
package bindinfo

import (
"unsafe"

"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
)
Expand Down Expand Up @@ -64,3 +67,19 @@ func newBindRecord(row chunk.Row) *BindRecord {
Collation: row.GetString(7),
}
}

// size calculates the memory size of a bind meta.
func (m *BindRecord) size() float64 {
res := len(m.OriginalSQL) + len(m.BindSQL) + len(m.Db) + len(m.Status) + 2*int(unsafe.Sizeof(m.CreateTime)) + len(m.Charset) + len(m.Collation)
return float64(res)
}

func (m *BindRecord) updateMetrics(scope string, inc bool) {
if inc {
metrics.BindMemoryUsage.WithLabelValues(scope, m.Status).Add(float64(m.size()))
metrics.BindTotalGauge.WithLabelValues(scope, m.Status).Inc()
} else {
metrics.BindMemoryUsage.WithLabelValues(scope, m.Status).Sub(float64(m.size()))
metrics.BindTotalGauge.WithLabelValues(scope, m.Status).Dec()
}
}
20 changes: 13 additions & 7 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -123,9 +124,10 @@ func (h *BindHandle) Update(fullLoad bool) (err error) {
continue
}

newCache.removeStaleBindMetas(hash, meta)
newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal)
if meta.Status == Using {
newCache[hash] = append(newCache[hash], meta)
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, meta.Status).Add(meta.size())
}
}
return nil
Expand Down Expand Up @@ -254,6 +256,7 @@ func (h *BindHandle) DropInvalidBindRecord() {

if time.Since(invalidBindRecord.droppedTime) > 6*time.Second {
delete(invalidBindRecordMap, key)
invalidBindRecord.bindRecord.updateMetrics(metrics.ScopeGlobal, false)
}
}
h.invalidBindRecordMap.Store(invalidBindRecordMap)
Expand All @@ -275,6 +278,7 @@ func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) {
bindRecord: invalidBindRecord,
}
h.invalidBindRecordMap.Store(newMap)
invalidBindRecord.updateMetrics(metrics.ScopeGlobal, true)
}

// Size return the size of bind info cache.
Expand Down Expand Up @@ -320,8 +324,9 @@ func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) {
// removed from the cache after this operation.
func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) {
newCache := h.bindInfo.Value.Load().(cache).copy()
newCache.removeStaleBindMetas(hash, meta)
newCache.removeStaleBindMetas(hash, meta, metrics.ScopeGlobal)
newCache[hash] = append(newCache[hash], meta)
meta.updateMetrics(metrics.ScopeGlobal, true)
h.bindInfo.Value.Store(newCache)
}

Expand All @@ -334,18 +339,19 @@ func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) {
h.bindInfo.Unlock()
}()

newCache.removeDeletedBindMeta(hash, meta)
newCache.removeDeletedBindMeta(hash, meta, metrics.ScopeGlobal)
}

// removeDeletedBindMeta removes all the BindMeta which originSQL and db are the same with the parameter's meta.
func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta) {
func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta, scope string) {
metas, ok := c[hash]
if !ok {
return
}

for i := len(metas) - 1; i >= 0; i-- {
if meta.isSame(meta) {
if metas[i].isSame(meta) {
metas[i].updateMetrics(scope, false)
metas = append(metas[:i], metas[i+1:]...)
if len(metas) == 0 {
delete(c, hash)
Expand All @@ -356,15 +362,15 @@ func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta) {
}

// removeStaleBindMetas removes all the stale BindMeta in the cache.
func (c cache) removeStaleBindMetas(hash string, meta *BindMeta) {
func (c cache) removeStaleBindMetas(hash string, meta *BindMeta, scope string) {
metas, ok := c[hash]
if !ok {
return
}

// remove stale bindMetas.
for i := len(metas) - 1; i >= 0; i-- {
if metas[i].isStale(meta) {
metas[i].updateMetrics(scope, false)
metas = append(metas[:i], metas[i+1:]...)
if len(metas) == 0 {
delete(c, hash)
Expand Down
15 changes: 13 additions & 2 deletions bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (

"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
)

Expand All @@ -38,8 +39,9 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle {
// removed from the cache after this operation.
func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) {
// Make sure there is only one goroutine writes the cache.
h.ch.removeStaleBindMetas(hash, meta)
h.ch.removeStaleBindMetas(hash, meta, metrics.ScopeSession)
h.ch[hash] = append(h.ch[hash], meta)
meta.updateMetrics(metrics.ScopeSession, true)
}

func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) {
Expand Down Expand Up @@ -74,7 +76,7 @@ func (h *SessionHandle) DropBindRecord(record *BindRecord) {
meta := &BindMeta{BindRecord: record}
meta.Status = deleted
hash := parser.DigestHash(record.OriginalSQL)
h.ch.removeDeletedBindMeta(hash, meta)
h.ch.removeDeletedBindMeta(hash, meta, metrics.ScopeSession)
h.appendBindMeta(hash, meta)
}

Expand All @@ -100,6 +102,15 @@ func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) {
return bindRecords
}

// Close closes the session handle.
func (h *SessionHandle) Close() {
for _, bindRecords := range h.ch {
for _, bindRecord := range bindRecords {
bindRecord.updateMetrics(metrics.ScopeSession, false)
}
}
}

// sessionBindInfoKeyType is a dummy type to avoid naming collision in context.
type sessionBindInfoKeyType int

Expand Down
3 changes: 1 addition & 2 deletions cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -357,10 +357,9 @@ func (t *tester) execute(query query) error {
gotBuf := t.buf.Bytes()[offset:]

buf := make([]byte, t.buf.Len()-offset)
if _, err = t.resultFD.ReadAt(buf, int64(offset)); err != nil {
if _, err = t.resultFD.ReadAt(buf, int64(offset)); !(err == nil || err == io.EOF) {
return errors.Trace(errors.Errorf("run \"%v\" at line %d err, we got \n%s\nbut read result err %s", st.Text(), query.Line, gotBuf, err))
}

if !bytes.Equal(gotBuf, buf) {
return errors.Trace(errors.Errorf("run \"%v\" at line %d err, we need:\n%s\nbut got:\n%s\n", query.Query, query.Line, buf, gotBuf))
}
Expand Down
6 changes: 3 additions & 3 deletions cmd/explaintest/r/access_path_selection.result
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@ IndexReader_6 3323.33 root index:IndexScan_5
└─IndexScan_5 3323.33 cop table:access_path_selection, index:a, b, range:[-inf,3), keep order:false, stats:pseudo
explain select a, b from access_path_selection where b < 3;
id count task operator info
IndexLookUp_10 3323.33 root
├─IndexScan_8 3323.33 cop table:access_path_selection, index:b, range:[-inf,3), keep order:false, stats:pseudo
└─TableScan_9 3323.33 cop table:access_path_selection, keep order:false, stats:pseudo
TableReader_7 3323.33 root data:Selection_6
└─Selection_6 3323.33 cop lt(test.access_path_selection.b, 3)
└─TableScan_5 10000.00 cop table:access_path_selection, range:[-inf,+inf], keep order:false, stats:pseudo
explain select a, b from access_path_selection where a < 3 and b < 3;
id count task operator info
IndexReader_11 1104.45 root index:Selection_10
Expand Down
35 changes: 35 additions & 0 deletions cmd/explaintest/r/black_list.result
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
use test;
drop table if exists t;
create table t (a int);
explain select * from t where a < 1;
id count task operator info
TableReader_7 3323.33 root data:Selection_6
└─Selection_6 3323.33 cop lt(test.t.a, 1)
└─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
insert into mysql.opt_rule_blacklist values('predicate_push_down');
admin reload opt_rule_blacklist;

explain select * from t where a < 1;
id count task operator info
Selection_5 8000.00 root lt(test.t.a, 1)
└─TableReader_7 10000.00 root data:TableScan_6
└─TableScan_6 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
delete from mysql.opt_rule_blacklist where name='predicate_push_down';
admin reload opt_rule_blacklist;

explain select * from t where a < 1;
id count task operator info
TableReader_7 3323.33 root data:Selection_6
└─Selection_6 3323.33 cop lt(test.t.a, 1)
└─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
insert into mysql.expr_pushdown_blacklist values('lt');
admin reload expr_pushdown_blacklist;

explain select * from t where a < 1;
id count task operator info
Selection_5 8000.00 root lt(test.t.a, 1)
└─TableReader_7 10000.00 root data:TableScan_6
└─TableScan_6 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
delete from mysql.expr_pushdown_blacklist where name='lt';
admin reload expr_pushdown_blacklist;

2 changes: 2 additions & 0 deletions cmd/explaintest/r/explain.result
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ Field Type Null Key Default Extra
id int(11) YES NULL
drop table if exists t;
create table t(id int primary key, a int, b int);
set session tidb_hashagg_partial_concurrency = 1;
set session tidb_hashagg_final_concurrency = 1;
explain select group_concat(a) from t group by id;
id count task operator info
StreamAgg_8 8000.00 root group by:col_1, funcs:group_concat(col_0, ",")
Expand Down
Loading

0 comments on commit c1c5c07

Please sign in to comment.