From 88bf9f7ba0243bd4a46f0cd2043afb3c3b7e4e13 Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 6 Aug 2018 11:57:46 +0800 Subject: [PATCH 01/87] admin: support prefix index in admin check table. (#7241) --- executor/admin_test.go | 10 ++++++++++ table/tables/index.go | 19 +++++++++++++------ util/admin/admin.go | 21 +++++++++++++++++++++ 3 files changed, 44 insertions(+), 6 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 8c6684821eaa4..01951190d4c77 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -459,4 +459,14 @@ func (s *testSuite) TestAdminCheckTable(c *C) { tk.MustExec(fmt.Sprintf("insert into test values (%d, %d);", i, i)) } tk.MustExec(`admin check table test;`) + + // Test prefix index. + tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t ( + ID CHAR(32) NOT NULL, + name CHAR(32) NOT NULL, + value CHAR(255), + INDEX indexIDname (ID(8),name(8)));`) + tk.MustExec(`INSERT INTO t VALUES ('keyword','urlprefix','text/ /text');`) + tk.MustExec(`admin check table t;`) } diff --git a/table/tables/index.go b/table/tables/index.go index 56b5f7951ee00..d7b71c051db5b 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -129,25 +129,32 @@ func (c *index) getIndexKeyBuf(buf []byte, defaultCap int) []byte { return make([]byte, 0, defaultCap) } -// truncateIndexValuesIfNeeded truncate the index values that be created that use only the leading part of column values. -func (c *index) truncateIndexValuesIfNeeded(indexedValues []types.Datum) []types.Datum { +// TruncateIndexValuesIfNeeded truncates the index values created using only the leading part of column values. +func TruncateIndexValuesIfNeeded(tblInfo *model.TableInfo, idxInfo *model.IndexInfo, indexedValues []types.Datum) []types.Datum { for i := 0; i < len(indexedValues); i++ { v := &indexedValues[i] if v.Kind() == types.KindString || v.Kind() == types.KindBytes { - ic := c.idxInfo.Columns[i] - colCharset := c.tblInfo.Columns[ic.Offset].Charset + ic := idxInfo.Columns[i] + colCharset := tblInfo.Columns[ic.Offset].Charset colValue := v.GetBytes() isUTF8Charset := colCharset == charset.CharsetUTF8 || colCharset == charset.CharsetUTF8MB4 + origKind := v.Kind() if isUTF8Charset { if ic.Length != types.UnspecifiedLength && utf8.RuneCount(colValue) > ic.Length { rs := bytes.Runes(colValue) truncateStr := string(rs[:ic.Length]) // truncate value and limit its length v.SetString(truncateStr) + if origKind == types.KindBytes { + v.SetBytes(v.GetBytes()) + } } } else if ic.Length != types.UnspecifiedLength && len(colValue) > ic.Length { // truncate value and limit its length v.SetBytes(colValue[:ic.Length]) + if origKind == types.KindString { + v.SetString(v.GetString()) + } } } } @@ -172,9 +179,9 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types. } } - // For string columns, indexes can be created that use only the leading part of column values, + // For string columns, indexes can be created using only the leading part of column values, // using col_name(length) syntax to specify an index prefix length. - indexedValues = c.truncateIndexValuesIfNeeded(indexedValues) + indexedValues = TruncateIndexValuesIfNeeded(c.tblInfo, c.idxInfo, indexedValues) key = c.getIndexKeyBuf(buf, len(c.prefix)+len(indexedValues)*9+9) key = append(key, []byte(c.prefix)...) key, err = codec.EncodeKey(sc, key, indexedValues...) diff --git a/util/admin/admin.go b/util/admin/admin.go index 63fd9c6ca60ef..4953148c3d7f0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" @@ -287,6 +288,24 @@ func getIndexFieldTypes(t table.Table, idx table.Index) ([]*types.FieldType, err return fieldTypes, nil } +// adjustDatumKind treats KindString as KindBytes. +func adjustDatumKind(vals1, vals2 []types.Datum) { + if len(vals1) != len(vals2) { + return + } + + for i, val1 := range vals1 { + val2 := vals2[i] + if val1.Kind() != val2.Kind() { + if (val1.Kind() == types.KindBytes || val1.Kind() == types.KindString) && + (val2.Kind() == types.KindBytes || val2.Kind() == types.KindString) { + vals1[i].SetBytes(val1.GetBytes()) + vals2[i].SetBytes(val2.GetBytes()) + } + } + } +} + func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { it, err := idx.SeekFirst(txn) if err != nil { @@ -316,6 +335,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table return errors.Trace(err) } vals2, err := rowWithCols(sessCtx, txn, t, h, cols) + vals2 = tables.TruncateIndexValuesIfNeeded(t.Meta(), idx.Meta(), vals2) if kv.ErrNotExist.Equal(err) { record := &RecordData{Handle: h, Values: vals1} err = errDateNotEqual.Gen("index:%#v != record:%#v", record, nil) @@ -323,6 +343,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table if err != nil { return errors.Trace(err) } + adjustDatumKind(vals1, vals2) if !reflect.DeepEqual(vals1, vals2) { record1 := &RecordData{Handle: h, Values: vals1} record2 := &RecordData{Handle: h, Values: vals2} From a7fc9c323ede9490761f91eb7815d6b93b6d955a Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 6 Aug 2018 13:07:06 +0800 Subject: [PATCH 02/87] distsql: disable read committed isolation level (#7280) read committed isolation may cause inconsistent read. Only Analyze request use read committed isolation level. --- ddl/reorg.go | 1 - distsql/request_builder.go | 15 ++++----------- distsql/request_builder_test.go | 2 +- executor/analyze.go | 3 --- executor/set.go | 8 -------- session/session.go | 4 ---- session/session_test.go | 2 +- store/tikv/lock_test.go | 16 ---------------- store/tikv/scan.go | 5 ++--- store/tikv/snapshot.go | 32 ++++++++++++++------------------ store/tikv/txn.go | 5 ----- 11 files changed, 22 insertions(+), 71 deletions(-) diff --git a/ddl/reorg.go b/ddl/reorg.go index e4b3116e8f32c..623ab8d2ecfc2 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -248,7 +248,6 @@ func (d *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl tab builder.Request.NotFillCache = true builder.Request.Priority = kv.PriorityLow - builder.Request.IsolationLevel = kv.SI kvReq, err := builder.Build() sctx := newContext(d.store) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 5e569a80d7bb3..5e5261bfb9a61 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -17,7 +17,6 @@ import ( "math" "github.com/juju/errors" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -125,15 +124,9 @@ func (builder *RequestBuilder) SetKeepOrder(order bool) *RequestBuilder { return builder } -func (builder *RequestBuilder) getIsolationLevel(sv *variable.SessionVars) kv.IsoLevel { - var isoLevel string - if sv.TxnIsolationLevelOneShot.State == 2 { - isoLevel = sv.TxnIsolationLevelOneShot.Value - } - if isoLevel == "" { - isoLevel, _ = sv.GetSystemVar(variable.TxnIsolation) - } - if isoLevel == ast.ReadCommitted { +func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel { + switch builder.Tp { + case kv.ReqTypeAnalyze: return kv.RC } return kv.SI @@ -155,7 +148,7 @@ func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int { // "Concurrency", "IsolationLevel", "NotFillCache". func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *RequestBuilder { builder.Request.Concurrency = sv.DistSQLScanConcurrency - builder.Request.IsolationLevel = builder.getIsolationLevel(sv) + builder.Request.IsolationLevel = builder.getIsolationLevel() builder.Request.NotFillCache = sv.StmtCtx.NotFillCache builder.Request.Priority = builder.getKVPriority(sv) return builder diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index d9bf8f261c541..bfd233318c5d9 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -507,7 +507,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) { KeepOrder: true, Desc: false, Concurrency: 15, - IsolationLevel: 0, + IsolationLevel: kv.RC, Priority: 1, NotFillCache: true, SyncLog: false, diff --git a/executor/analyze.go b/executor/analyze.go index 2c20de123867f..d9b08ece4fd15 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -20,7 +20,6 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" @@ -176,7 +175,6 @@ func (e *AnalyzeIndexExec) open() error { SetKeepOrder(true). Build() kvReq.Concurrency = e.concurrency - kvReq.IsolationLevel = kv.RC ctx := context.TODO() e.result, err = distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars) if err != nil { @@ -291,7 +289,6 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe SetAnalyzeRequest(e.analyzePB). SetKeepOrder(e.keepOrder). Build() - kvReq.IsolationLevel = kv.RC kvReq.Concurrency = e.concurrency if err != nil { return nil, errors.Trace(err) diff --git a/executor/set.go b/executor/set.go index 97509d766a458..29f783a96f47a 100644 --- a/executor/set.go +++ b/executor/set.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/terror" @@ -181,13 +180,6 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e log.Infof("con:%d %s=%s", sessionVars.ConnectionID, name, valStr) } - if name == variable.TxnIsolation { - isoLevel, _ := sessionVars.GetSystemVar(variable.TxnIsolation) - if isoLevel == ast.ReadCommitted { - e.ctx.Txn().SetOption(kv.IsolationLevel, kv.RC) - } - } - return nil } diff --git a/session/session.go b/session/session.go index 981ab1412caea..d76313e581abd 100644 --- a/session/session.go +++ b/session/session.go @@ -1351,10 +1351,6 @@ func (s *session) ActivePendingTxn() error { return errors.Trace(err) } s.sessionVars.TxnCtx.StartTS = s.txn.StartTS() - isoLevel, _ := s.sessionVars.GetSystemVar(variable.TxnIsolation) - if isoLevel == ast.ReadCommitted { - s.txn.SetOption(kv.IsolationLevel, kv.RC) - } return nil } diff --git a/session/session_test.go b/session/session_test.go index fa956eb36530a..2179a89e1c7a2 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2000,7 +2000,7 @@ func (s *testSessionSuite) TestSetTransactionIsolationOneShot(c *C) { // Check isolation level is set to read committed. ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, kv.RC) + c.Assert(req.IsolationLevel, Equals, kv.SI) }) tk.Se.Execute(ctx, "select * from t where k = 1") diff --git a/store/tikv/lock_test.go b/store/tikv/lock_test.go index e3177dafc4ce0..f544053f6dbb6 100644 --- a/store/tikv/lock_test.go +++ b/store/tikv/lock_test.go @@ -181,22 +181,6 @@ func (s *testLockSuite) TestGetTxnStatus(c *C) { c.Assert(status.IsCommitted(), IsFalse) } -func (s *testLockSuite) TestRC(c *C) { - s.putKV(c, []byte("key"), []byte("v1")) - - txn, err := s.store.Begin() - c.Assert(err, IsNil) - txn.Set([]byte("key"), []byte("v2")) - s.prewriteTxn(c, txn.(*tikvTxn)) - - txn2, err := s.store.Begin() - c.Assert(err, IsNil) - txn2.SetOption(kv.IsolationLevel, kv.RC) - val, err := txn2.Get([]byte("key")) - c.Assert(err, IsNil) - c.Assert(string(val), Equals, "v1") -} - func (s *testLockSuite) prewriteTxn(c *C, txn *tikvTxn) { committer, err := newTwoPhaseCommitter(txn, 0) c.Assert(err, IsNil) diff --git a/store/tikv/scan.go b/store/tikv/scan.go index d8e57dc1d4020..0ccf297666ee6 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -146,9 +146,8 @@ func (s *Scanner) getData(bo *Backoffer) error { Version: s.startTS(), }, Context: pb.Context{ - IsolationLevel: pbIsolationLevel(s.snapshot.isolationLevel), - Priority: s.snapshot.priority, - NotFillCache: s.snapshot.notFillCache, + Priority: s.snapshot.priority, + NotFillCache: s.snapshot.notFillCache, }, } resp, err := sender.SendReq(bo, req, loc.Region, ReadTimeoutMedium) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 2c91c3bd1d5b6..986dd2dd58943 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -42,13 +42,12 @@ const ( // tikvSnapshot implements the kv.Snapshot interface. type tikvSnapshot struct { - store *tikvStore - version kv.Version - isolationLevel kv.IsoLevel - priority pb.CommandPri - notFillCache bool - syncLog bool - vars *kv.Variables + store *tikvStore + version kv.Version + priority pb.CommandPri + notFillCache bool + syncLog bool + vars *kv.Variables } var snapshotGP = gp.New(time.Minute) @@ -56,11 +55,10 @@ var snapshotGP = gp.New(time.Minute) // newTiKVSnapshot creates a snapshot of an TiKV store. func newTiKVSnapshot(store *tikvStore, ver kv.Version) *tikvSnapshot { return &tikvSnapshot{ - store: store, - version: ver, - isolationLevel: kv.SI, - priority: pb.CommandPri_Normal, - vars: kv.DefaultVars, + store: store, + version: ver, + priority: pb.CommandPri_Normal, + vars: kv.DefaultVars, } } @@ -151,9 +149,8 @@ func (s *tikvSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, coll Version: s.version.Ver, }, Context: pb.Context{ - Priority: s.priority, - IsolationLevel: pbIsolationLevel(s.isolationLevel), - NotFillCache: s.notFillCache, + Priority: s.priority, + NotFillCache: s.notFillCache, }, } resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutMedium) @@ -233,9 +230,8 @@ func (s *tikvSnapshot) get(bo *Backoffer, k kv.Key) ([]byte, error) { Version: s.version.Ver, }, Context: pb.Context{ - Priority: s.priority, - IsolationLevel: pbIsolationLevel(s.isolationLevel), - NotFillCache: s.notFillCache, + Priority: s.priority, + NotFillCache: s.notFillCache, }, } for { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 27bd026f88b59..a39a0d110ceaa 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -144,8 +144,6 @@ func (txn *tikvTxn) Delete(k kv.Key) error { func (txn *tikvTxn) SetOption(opt kv.Option, val interface{}) { txn.us.SetOption(opt, val) switch opt { - case kv.IsolationLevel: - txn.snapshot.isolationLevel = val.(kv.IsoLevel) case kv.Priority: txn.snapshot.priority = kvPriorityToCommandPri(val.(int)) case kv.NotFillCache: @@ -157,9 +155,6 @@ func (txn *tikvTxn) SetOption(opt kv.Option, val interface{}) { func (txn *tikvTxn) DelOption(opt kv.Option) { txn.us.DelOption(opt) - if opt == kv.IsolationLevel { - txn.snapshot.isolationLevel = kv.SI - } } func (txn *tikvTxn) Commit(ctx context.Context) error { From 44e6c3c2e110238dd9ff834a24ec3e2cb0dc6062 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 6 Aug 2018 17:39:33 +0800 Subject: [PATCH 03/87] plan, stats: fix inconsistent row count estimation (#7233) --- cmd/explaintest/r/explain_easy.result | 21 +++++++------ cmd/explaintest/r/explain_easy_stats.result | 11 ++++--- executor/analyze.go | 17 ++++++++-- plan/cbo_test.go | 35 +++++++++++++++++++++ plan/logical_plans.go | 10 ++++++ statistics/selectivity.go | 16 +++++++--- 6 files changed, 87 insertions(+), 23 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 51cdef794fe6b..2e096f0e4cfc1 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -112,13 +112,14 @@ MemTableScan_4 10000.00 root explain select c2 = (select c2 from t2 where t1.c1 = t2.c1 order by c1 limit 1) from t1; id count task operator info Projection_12 10000.00 root eq(test.t1.c2, test.t2.c2) -└─Apply_14 10000.00 root left outer join, inner:Limit_22 +└─Apply_14 10000.00 root left outer join, inner:Limit_21 ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo - └─Limit_22 1.00 root offset:0, count:1 - └─IndexLookUp_47 0.00 root - ├─IndexScan_45 0.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo - └─TableScan_46 0.00 cop table:t2, keep order:false, stats:pseudo + └─Limit_21 1.00 root offset:0, count:1 + └─IndexLookUp_43 1.00 root + ├─Limit_42 1.00 cop offset:0, count:1 + │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo + └─TableScan_41 1.00 cop table:t2, keep order:false explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 @@ -286,8 +287,8 @@ Projection_11 10000.00 root 9_aux_0 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) └─IndexJoin_32 10000.00 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a - ├─IndexReader_36 10.00 root index:IndexScan_35 - │ └─IndexScan_35 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo + ├─IndexReader_36 10000.00 root index:IndexScan_35 + │ └─IndexScan_35 10000.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo └─TableReader_31 10.00 root data:TableScan_30 └─TableScan_30 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t; @@ -298,9 +299,9 @@ Projection_11 10000.00 root 9_aux_0 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) └─IndexJoin_33 10000.00 root inner join, inner:TableReader_32, outer key:s.c, inner key:t1.a - ├─IndexLookUp_38 10.00 root - │ ├─IndexScan_36 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo - │ └─TableScan_37 10.00 cop table:t, keep order:false, stats:pseudo + ├─IndexLookUp_38 10000.00 root + │ ├─IndexScan_36 10000.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo + │ └─TableScan_37 10000.00 cop table:t, keep order:false, stats:pseudo └─TableReader_32 10.00 root data:TableScan_31 └─TableScan_31 10.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo drop table if exists t; diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 3e314cb6c1054..740fa7b9fea4b 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -101,13 +101,14 @@ MemTableScan_4 10000.00 root explain select c2 = (select c2 from t2 where t1.c1 = t2.c1 order by c1 limit 1) from t1; id count task operator info Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2) -└─Apply_14 1999.00 root left outer join, inner:Limit_22 +└─Apply_14 1999.00 root left outer join, inner:Limit_21 ├─TableReader_16 1999.00 root data:TableScan_15 │ └─TableScan_15 1999.00 cop table:t1, range:[-inf,+inf], keep order:false - └─Limit_22 1.00 root offset:0, count:1 - └─IndexLookUp_47 0.00 root - ├─IndexScan_45 0.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true - └─TableScan_46 0.00 cop table:t2, keep order:false + └─Limit_21 1.00 root offset:0, count:1 + └─IndexLookUp_43 1.00 root + ├─Limit_42 1.00 cop offset:0, count:1 + │ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true + └─TableScan_41 1.00 cop table:t2, keep order:false explain select * from t1 order by c1 desc limit 1; id count task operator info Limit_10 1.00 root offset:0, count:1 diff --git a/executor/analyze.go b/executor/analyze.go index d9b08ece4fd15..0c172d38226a0 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -42,11 +42,12 @@ type AnalyzeExec struct { tasks []*analyzeTask } +var maxBucketSize = int64(256) + const ( maxSampleSize = 10000 maxRegionSampleSize = 1000 maxSketchSize = 10000 - maxBucketSize = 256 defaultCMSketchDepth = 5 defaultCMSketchWidth = 2048 ) @@ -210,7 +211,7 @@ func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statis if err != nil { return nil, nil, errors.Trace(err) } - hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, statistics.HistogramFromProto(resp.Hist), maxBucketSize) + hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, statistics.HistogramFromProto(resp.Hist), int(maxBucketSize)) if err != nil { return nil, nil, errors.Trace(err) } @@ -338,7 +339,7 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms [] } sc := e.ctx.GetSessionVars().StmtCtx if e.pkInfo != nil { - pkHist, err = statistics.MergeHistograms(sc, pkHist, statistics.HistogramFromProto(resp.PkHist), maxBucketSize) + pkHist, err = statistics.MergeHistograms(sc, pkHist, statistics.HistogramFromProto(resp.PkHist), int(maxBucketSize)) if err != nil { return nil, nil, errors.Trace(err) } @@ -373,3 +374,13 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms [] } return hists, cms, nil } + +// SetMaxBucketSizeForTest sets the `maxBucketSize`. +func SetMaxBucketSizeForTest(size int64) { + maxBucketSize = size +} + +// GetMaxBucketSizeForTest gets the `maxBucketSize`. +func GetMaxBucketSizeForTest() int64 { + return maxBucketSize +} diff --git a/plan/cbo_test.go b/plan/cbo_test.go index 5960960e12c15..3afcdafb25922 100644 --- a/plan/cbo_test.go +++ b/plan/cbo_test.go @@ -21,6 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/plan" "github.com/pingcap/tidb/session" @@ -620,6 +621,40 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { )) } +func (s *testAnalyzeSuite) TestInconsistentEstimation(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 t(a int, b int, c int, index ab(a,b), index ac(a,c))") + tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)") + for i := 0; i < 10; i++ { + tk.MustExec("insert into t values (5,5,5), (10,10,10)") + } + origin := executor.GetMaxBucketSizeForTest() + defer func() { executor.SetMaxBucketSizeForTest(origin) }() + executor.SetMaxBucketSizeForTest(2) + tk.MustExec("analyze table t") + // Force using the histogram to estimate. + tk.MustExec("update mysql.stats_histograms set stats_ver = 0") + dom.StatsHandle().Clear() + dom.StatsHandle().Update(dom.InfoSchema()) + // Using the histogram (a, b) to estimate `a = 5` will get 1.22, while using the CM Sketch to estimate + // the `a = 5 and c = 5` will get 10, it is not consistent. + tk.MustQuery("explain select * from t use index(ab) where a = 5 and c = 5"). + Check(testkit.Rows( + "IndexLookUp_8 10.00 root ", + "├─IndexScan_5 12.50 cop table:t, index:a, b, range:[5,5], keep order:false", + "└─Selection_7 10.00 cop eq(test.t.c, 5)", + " └─TableScan_6 12.50 cop table:t, keep order:false", + )) +} + func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { store, err := mockstore.NewMockTikvStore() if err != nil { diff --git a/plan/logical_plans.go b/plan/logical_plans.go index eabbd4ceeac47..cbc659de658e4 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -393,6 +393,11 @@ func (ds *DataSource) deriveTablePathStats(path *accessPath) (bool, error) { return false, errors.Trace(err) } path.countAfterAccess, err = ds.statisticTable.GetRowCountByIntColumnRanges(sc, pkCol.ID, path.ranges) + // If the `countAfterAccess` is less than `stats.count`, there must be some inconsistent stats info. + // We prefer the `stats.count` because it could use more stats info to calculate the selectivity. + if path.countAfterAccess < ds.stats.count { + path.countAfterAccess = math.Min(ds.stats.count/selectionFactor, float64(ds.statisticTable.Count)) + } // Check whether the primary key is covered by point query. noIntervalRange := true for _, ran := range path.ranges { @@ -443,6 +448,11 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { path.countAfterAccess = ds.statisticTable.PseudoAvgCountPerValue() } } + // If the `countAfterAccess` is less than `stats.count`, there must be some inconsistent stats info. + // We prefer the `stats.count` because it could use more stats info to calculate the selectivity. + if path.countAfterAccess < ds.stats.count { + path.countAfterAccess = math.Min(ds.stats.count/selectionFactor, float64(ds.statisticTable.Count)) + } if path.indexFilters != nil { selectivity, err := ds.statisticTable.Selectivity(ds.ctx, path.indexFilters) if err != nil { diff --git a/statistics/selectivity.go b/statistics/selectivity.go index a98bfe4d51efa..79260584adbba 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -35,6 +35,8 @@ type exprSet struct { mask int64 // ranges contains all the ranges we got. ranges []*ranger.Range + // numCols is the number of columns contained in the index or column(which is always 1). + numCols int } // The type of the exprSet. @@ -177,7 +179,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp if err != nil { return 0, errors.Trace(err) } - sets = append(sets, &exprSet{tp: colType, ID: col.ID, mask: maskCovered, ranges: ranges}) + sets = append(sets, &exprSet{tp: colType, ID: col.ID, mask: maskCovered, ranges: ranges, numCols: 1}) if mysql.HasPriKeyFlag(colInfo.Info.Flag) { sets[len(sets)-1].tp = pkType } @@ -190,7 +192,7 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp if err != nil { return 0, errors.Trace(err) } - sets = append(sets, &exprSet{tp: indexType, ID: idxInfo.ID, mask: maskCovered, ranges: ranges}) + sets = append(sets, &exprSet{tp: indexType, ID: idxInfo.ID, mask: maskCovered, ranges: ranges, numCols: len(idxInfo.Info.Columns)}) } } sets = getUsableSetsByGreedy(sets) @@ -254,7 +256,7 @@ func getUsableSetsByGreedy(sets []*exprSet) (newBlocks []*exprSet) { mask := int64(math.MaxInt64) for { // Choose the index that covers most. - bestID, bestCount, bestTp := -1, 0, colType + bestID, bestCount, bestTp, bestNumCols := -1, 0, colType, 0 for i, set := range sets { set.mask &= mask bits := popCount(set.mask) @@ -262,8 +264,12 @@ func getUsableSetsByGreedy(sets []*exprSet) (newBlocks []*exprSet) { if bits == 0 { continue } - if (bestTp == colType && set.tp < colType) || bestCount < bits { - bestID, bestCount, bestTp = i, bits, set.tp + // We greedy select the stats info based on: + // (1): The stats type, always prefer the primary key or index. + // (2): The number of expression that it covers, the more the better. + // (3): The number of columns that it contains, the less the better. + if (bestTp == colType && set.tp != colType) || bestCount < bits || (bestCount == bits && bestNumCols > set.numCols) { + bestID, bestCount, bestTp, bestNumCols = i, bits, set.tp, set.numCols } } if bestCount == 0 { From 326baacb933f76d0112afd6f82f6c65b35c909cd Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 6 Aug 2018 20:39:31 +0800 Subject: [PATCH 04/87] ddl: batch check the constrains when we add a unique-index. (#7132) --- ddl/db_integration_test.go | 16 +++++++ ddl/db_test.go | 1 - ddl/index.go | 93 +++++++++++++++++++++++++++++++++++--- 3 files changed, 103 insertions(+), 7 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 632d366dc9e6e..4caab8c4fba8e 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -123,6 +123,22 @@ func (s *testIntegrationSuite) TestCreateTableIfNotExists(c *C) { c.Assert(terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), IsTrue) } +func (s *testIntegrationSuite) TestUniquekeyNullValue(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("USE test") + + tk.MustExec("create table t(a int primary key, b varchar(255))") + + tk.MustExec("insert into t values(1, NULL)") + tk.MustExec("insert into t values(2, NULL)") + tk.MustExec("alter table t add unique index b(b);") + res := tk.MustQuery("select count(*) from t use index(b);") + res.Check(testkit.Rows("2")) + tk.MustExec("admin check table t") + tk.MustExec("admin check index t b") +} + func (s *testIntegrationSuite) TestEndIncluded(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/db_test.go b/ddl/db_test.go index ba363b6d434e3..c880f6ed3088b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -384,7 +384,6 @@ LOOP: s.mustExec(c, "delete from t1 where c1 = ?", i+10) } sessionExec(c, s.store, "create index c3_index on t1 (c3)") - s.mustExec(c, "drop table t1") } diff --git a/ddl/index.go b/ddl/index.go index ff7b3ad47150e..d6b1b5394158d 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -454,6 +454,7 @@ type indexRecord struct { handle int64 key []byte // It's used to lock a record. Record it to reduce the encoding time. vals []types.Datum // It's the index values. + skip bool // skip indicates that the index key is already exists, we should not add it. } type addIndexWorker struct { @@ -468,9 +469,13 @@ type addIndexWorker struct { colFieldMap map[int64]*types.FieldType closed bool - defaultVals []types.Datum // It's used to reduce the number of new slice. - idxRecords []*indexRecord // It's used to reduce the number of new slice. - rowMap map[int64]types.Datum // It's the index column values map. It is used to reduce the number of making map. + // The following attributes are used to reduce memory allocation. + defaultVals []types.Datum + idxRecords []*indexRecord + rowMap map[int64]types.Datum + idxKeyBufs [][]byte + batchCheckKeys []kv.Key + distinctCheckFlags []bool } type reorgIndexTask struct { @@ -502,7 +507,6 @@ func newAddIndexWorker(sessCtx sessionctx.Context, worker *worker, id int, t tab index: index, table: t, colFieldMap: colFieldMap, - defaultVals: make([]types.Datum, len(t.Cols())), rowMap: make(map[int64]types.Datum, len(colFieldMap)), } @@ -632,6 +636,71 @@ func (w *addIndexWorker) logSlowOperations(elapsed time.Duration, slowMsg string } } +func (w *addIndexWorker) initBatchCheckBufs(batchCount int) { + if len(w.idxKeyBufs) < batchCount { + w.idxKeyBufs = make([][]byte, batchCount) + } + + w.batchCheckKeys = w.batchCheckKeys[:0] + w.distinctCheckFlags = w.distinctCheckFlags[:0] +} + +func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*indexRecord) error { + idxInfo := w.index.Meta() + if !idxInfo.Unique { + // non-unique key need not to check, just overwrite it, + // because in most case, backfilling indices is not exists. + return nil + } + + w.initBatchCheckBufs(len(idxRecords)) + stmtCtx := w.sessCtx.GetSessionVars().StmtCtx + for i, record := range idxRecords { + idxKey, distinct, err := w.index.GenIndexKey(stmtCtx, record.vals, record.handle, w.idxKeyBufs[i]) + if err != nil { + return errors.Trace(err) + } + // save the buffer to reduce memory allocations. + w.idxKeyBufs[i] = idxKey + + w.batchCheckKeys = append(w.batchCheckKeys, idxKey) + w.distinctCheckFlags = append(w.distinctCheckFlags, distinct) + } + + batchVals, err := kv.BatchGetValues(txn, w.batchCheckKeys) + if err != nil { + return errors.Trace(err) + } + + // 1. unique-key is duplicate and the handle is equal, skip it. + // 2. unique-key is duplicate and the handle is not equal, return duplicate error. + // 3. non-unique-key is duplicate, skip it. + for i, key := range w.batchCheckKeys { + if val, found := batchVals[string(key)]; found { + if w.distinctCheckFlags[i] { + handle, err1 := tables.DecodeHandle(val) + if err1 != nil { + return errors.Trace(err1) + } + + if handle != idxRecords[i].handle { + return errors.Trace(kv.ErrKeyExists) + } + } + idxRecords[i].skip = true + } else { + // The keys in w.batchCheckKeys also maybe duplicate, + // so we need to backfill the not found key into `batchVals` map. + if w.distinctCheckFlags[i] { + batchVals[string(key)] = tables.EncodeHandle(idxRecords[i].handle) + } + } + } + // Constrains is already checked. + stmtCtx.BatchCheck = true + return nil +} + // backfillIndexInTxn will backfill table index in a transaction, lock corresponding rowKey, if the value of rowKey is changed, // indicate that index columns values may changed, index is not allowed to be added, so the txn will rollback and retry. // backfillIndexInTxn will add w.batchCnt indices once, default value of w.batchCnt is 128. @@ -651,15 +720,27 @@ func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (nextHan return errors.Trace(err) } + err = w.batchCheckUniqueKey(txn, idxRecords) + if err != nil { + return errors.Trace(err) + } + for _, idxRecord := range idxRecords { + scanCount++ + // The index is already exists, we skip it, no needs to backfill it. + // The following update, delete, insert on these rows, TiDB can handle it correctly. + if idxRecord.skip { + continue + } + + // Lock the row key to notify us that someone delete or update the row, + // then we should not backfill the index of it, otherwise the adding index is redundant. err := txn.LockKeys(idxRecord.key) if err != nil { return errors.Trace(err) } - scanCount++ // Create the index. - // TODO: backfill unique-key will check constraint every row, we can speed up this case by using batch check. handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, idxRecord.handle) if err != nil { if kv.ErrKeyExists.Equal(err) && idxRecord.handle == handle { From 5a0fd2db46819a1eb27eca2cbcfaf1ca537bf5a2 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 6 Aug 2018 21:32:29 +0800 Subject: [PATCH 05/87] ddl: GetDDLJobs should return a sort job. (#7254) --- util/admin/admin.go | 22 +++++++++++++++--- util/admin/admin_test.go | 50 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 69 insertions(+), 3 deletions(-) diff --git a/util/admin/admin.go b/util/admin/admin.go index 4953148c3d7f0..efc2a60ebf1df 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -17,6 +17,7 @@ import ( "fmt" "io" "reflect" + "sort" "github.com/juju/errors" "github.com/pingcap/tidb/kv" @@ -136,8 +137,7 @@ func getDDLJobsInQueue(t *meta.Meta, jobListKey meta.JobListKeyType) ([]*model.J return jobs, nil } -// GetDDLJobs returns all DDL jobs. -// TODO: Sort jobs. +// GetDDLJobs get all DDL jobs and sorts jobs by job.ID. func GetDDLJobs(txn kv.Transaction) ([]*model.Job, error) { t := meta.NewMeta(txn) generalJobs, err := getDDLJobsInQueue(t, meta.DefaultJobListKey) @@ -148,7 +148,23 @@ func GetDDLJobs(txn kv.Transaction) ([]*model.Job, error) { if err != nil { return nil, errors.Trace(err) } - return append(generalJobs, addIdxJobs...), nil + jobs := append(generalJobs, addIdxJobs...) + sort.Sort(jobArray(jobs)) + return jobs, nil +} + +type jobArray []*model.Job + +func (v jobArray) Len() int { + return len(v) +} + +func (v jobArray) Less(i, j int) bool { + return v[i].ID < v[j].ID +} + +func (v jobArray) Swap(i, j int) { + v[i], v[j] = v[j], v[i] } // MaxHistoryJobs is exported for testing. diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index a1bab70c4806a..7dc87d33db35f 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -121,6 +121,56 @@ func (s *testSuite) TestGetDDLJobs(c *C) { c.Assert(err, IsNil) } +func isJobsSorted(jobs []*model.Job) bool { + if len(jobs) <= 1 { + return true + } + for i := 1; i < len(jobs); i++ { + if jobs[i].ID <= jobs[i-1].ID { + return false + } + } + return true +} + +func enQueueDDLJobs(c *C, t *meta.Meta, jobType model.ActionType, start, end int) { + for i := start; i < end; i++ { + job := &model.Job{ + ID: int64(i), + SchemaID: 1, + Type: jobType, + } + err := t.EnQueueDDLJob(job) + c.Assert(err, IsNil) + } +} + +func (s *testSuite) TestGetDDLJobsIsSort(c *C) { + txn, err := s.store.Begin() + c.Assert(err, IsNil) + + // insert 5 drop table jobs to DefaultJobListKey queue + t := meta.NewMeta(txn) + enQueueDDLJobs(c, t, model.ActionDropTable, 10, 15) + + // insert 5 create table jobs to DefaultJobListKey queue + enQueueDDLJobs(c, t, model.ActionCreateTable, 0, 5) + + // insert add index jobs to AddIndexJobListKey queue + t = meta.NewMeta(txn, meta.AddIndexJobListKey) + enQueueDDLJobs(c, t, model.ActionAddIndex, 5, 10) + + currJobs, err := GetDDLJobs(txn) + c.Assert(err, IsNil) + c.Assert(currJobs, HasLen, 15) + + isSort := isJobsSorted(currJobs) + c.Assert(isSort, Equals, true) + + err = txn.Rollback() + c.Assert(err, IsNil) +} + func (s *testSuite) TestCancelJobs(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) From 2225e0126425851059f4450e8207e801021ea411 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Mon, 6 Aug 2018 21:22:33 -0500 Subject: [PATCH 06/87] ddl:add constraints check when creating table partition (#7164) * ddl: add unique index check when creating table partition --- ddl/db_test.go | 192 +++++++++++++++++++++++++++++++++++++++++++++++ ddl/ddl.go | 4 + ddl/ddl_api.go | 23 +++--- ddl/partition.go | 63 ++++++++++++++++ 4 files changed, 272 insertions(+), 10 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index c880f6ed3088b..c792ee826060f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2905,6 +2905,198 @@ func (s *testDBSuite) TestTruncatePartitionAndDropTable(c *C) { s.testErrorCode(c, "select * from t4;", tmysql.ErrNoSuchTable) } +func (s *testDBSuite) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.tk.MustExec("use test;") + s.tk.MustExec("set @@session.tidb_enable_table_partition=1;") + s.tk.MustExec("drop table if exists part1;") + s.tk.MustExec(`create table part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1, col2) + ) + partition by range( col1 + col2 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );`) + + s.tk.MustExec("drop table if exists part2;") + s.tk.MustExec(`create table part2 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1, col2, col3), + unique key (col3) + ) + partition by range( col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );`) + + s.tk.MustExec("drop table if exists part3;") + s.tk.MustExec(`create table part3 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col2) + ) + partition by range( col1 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );`) + + s.tk.MustExec("drop table if exists part4;") + s.tk.MustExec(`create table part4 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col2), + unique key(col2) + ) + partition by range( year(col2) ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );`) + + s.tk.MustExec("drop table if exists part5;") + s.tk.MustExec(`create table part5 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col2, col4), + unique key(col2, col1) + ) + partition by range( col1 + col2 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );`) + + s.tk.MustExec("drop table if exists Part1;") + sql1 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1, col2) + ) + partition by range( col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql1, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql2 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1), + unique key (col3) + ) + partition by range( col1 + col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql2, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql3 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1), + unique key (col3) + ) + partition by range( col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql3, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql4 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + unique key (col1, col2, col3), + unique key (col3) + ) + partition by range( col1 + col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql4, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql5 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col2) + ) + partition by range( col3 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql5, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql6 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col3), + unique key(col2) + ) + partition by range( year(col2) ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql6, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists Part1;") + sql7 := `create table Part1 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + primary key(col1, col3, col4), + unique key(col2, col1) + ) + partition by range( col1 + col2 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql7, tmysql.ErrUniqueKeyNeedAllFieldsInPf) + + s.tk.MustExec("drop table if exists part6;") + sql8 := `create table part6 ( + col1 int not null, + col2 date not null, + col3 int not null, + col4 int not null, + col5 int not null, + unique key(col1, col2), + unique key(col1, col3) + ) + partition by range( col1 + col2 ) ( + partition p1 values less than (11), + partition p2 values less than (15) + );` + s.testErrorCode(c, sql8, tmysql.ErrUniqueKeyNeedAllFieldsInPf) +} + func (s *testDBSuite) TestPartitionAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl.go b/ddl/ddl.go index 5d46a51ac18c7..de0a0894359a9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -186,6 +186,8 @@ var ( ErrPartitionFunctionIsNotAllowed = terror.ClassDDL.New(codePartitionFunctionIsNotAllowed, mysql.MySQLErrName[mysql.ErrPartitionFunctionIsNotAllowed]) // ErrPartitionFuncNotAllowed returns partition function returns the wrong type. ErrPartitionFuncNotAllowed = terror.ClassDDL.New(codeErrPartitionFuncNotAllowed, mysql.MySQLErrName[mysql.ErrPartitionFuncNotAllowed]) + // ErrUniqueKeyNeedAllFieldsInPf returns must include all columns in the table's partitioning function. + ErrUniqueKeyNeedAllFieldsInPf = terror.ClassDDL.New(codeUniqueKeyNeedAllFieldsInPf, mysql.MySQLErrName[mysql.ErrUniqueKeyNeedAllFieldsInPf]) ) // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. @@ -595,6 +597,7 @@ const ( codePartitionFunctionIsNotAllowed = terror.ErrCode(mysql.ErrPartitionFunctionIsNotAllowed) codeErrPartitionFuncNotAllowed = terror.ErrCode(mysql.ErrPartitionFuncNotAllowed) codeErrFieldTypeNotAllowedAsPartitionField = terror.ErrCode(mysql.ErrFieldTypeNotAllowedAsPartitionField) + codeUniqueKeyNeedAllFieldsInPf = terror.ErrCode(mysql.ErrUniqueKeyNeedAllFieldsInPf) codePrimaryCantHaveNull = terror.ErrCode(mysql.ErrPrimaryCantHaveNull) ) @@ -640,6 +643,7 @@ func init() { codePartitionFunctionIsNotAllowed: mysql.ErrPartitionFunctionIsNotAllowed, codeErrPartitionFuncNotAllowed: mysql.ErrPartitionFuncNotAllowed, codeErrFieldTypeNotAllowedAsPartitionField: mysql.ErrFieldTypeNotAllowedAsPartitionField, + codeUniqueKeyNeedAllFieldsInPf: mysql.ErrUniqueKeyNeedAllFieldsInPf, codePrimaryCantHaveNull: mysql.ErrPrimaryCantHaveNull, } terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 34117c01cd73c..5ae069cb0232b 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -896,24 +896,27 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e } if pi != nil { - err = checkPartitionNameUnique(tbInfo, pi) - if err != nil { + if err = checkPartitionNameUnique(tbInfo, pi); err != nil { return errors.Trace(err) } - err = checkCreatePartitionValue(pi) - if err != nil { + + if err = checkCreatePartitionValue(pi); err != nil { return errors.Trace(err) } - err = checkAddPartitionTooManyPartitions(len(pi.Definitions)) - if err != nil { + + if err = checkAddPartitionTooManyPartitions(len(pi.Definitions)); err != nil { return errors.Trace(err) } - err = checkPartitionFuncValid(s.Partition.Expr) - if err != nil { + + if err = checkPartitionFuncValid(s.Partition.Expr); err != nil { return errors.Trace(err) } - err = checkPartitionFuncType(ctx, s, cols, tbInfo) - if err != nil { + + if err = checkPartitionFuncType(ctx, s, cols, tbInfo); err != nil { + return errors.Trace(err) + } + + if err = checkRangePartitioningKeysConstraints(ctx, s, tbInfo, newConstraints); err != nil { return errors.Trace(err) } tbInfo.Partition = pi diff --git a/ddl/partition.go b/ddl/partition.go index a6b841af6b64c..f9eaac5192fca 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -32,6 +32,7 @@ import ( const ( partitionMaxValue = "MAXVALUE" + primarykey = "PRIMARY KEY" ) // buildTablePartitionInfo builds partition info and checks for some errors. @@ -280,3 +281,65 @@ func getPartitionIDs(table *model.TableInfo) []int64 { } return partitionIDs } + +// checkRangePartitioningKeysConstraints checks that the range partitioning key is included in the table constraint. +func checkRangePartitioningKeysConstraints(ctx sessionctx.Context, s *ast.CreateTableStmt, tblInfo *model.TableInfo, constraints []*ast.Constraint) error { + // Returns directly if there is no constraint in the partition table. + if len(constraints) == 0 { + return nil + } + + // Extract the column names in table constraints to []map[string]struct{}. + consColNames := extractConstraintsColumnNames(constraints) + + // Parse partitioning key, extract the column names in the partitioning key to slice. + buf := new(bytes.Buffer) + s.Partition.Expr.Format(buf) + var partkeys []string + e, err := expression.ParseSimpleExpr(ctx, buf.String(), tblInfo) + if err != nil { + return errors.Trace(err) + } + cols := expression.ExtractColumns(e) + for _, col := range cols { + partkeys = append(partkeys, col.ColName.L) + } + + // Checks that the partitioning key is included in the constraint. + for _, con := range consColNames { + // Every unique key on the table must use every column in the table's partitioning expression. + // See https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations-partitioning-keys-unique-keys.html. + if !checkConstraintIncludePartKey(partkeys, con) { + return ErrUniqueKeyNeedAllFieldsInPf.GenByArgs(primarykey) + } + } + return nil +} + +// extractConstraintsColumnNames extract the column names in table constraints to []map[string]struct{}. +func extractConstraintsColumnNames(cons []*ast.Constraint) []map[string]struct{} { + var constraints []map[string]struct{} + for _, v := range cons { + if v.Tp == ast.ConstraintUniq || v.Tp == ast.ConstraintPrimaryKey { + uniKeys := make(map[string]struct{}) + for _, key := range v.Keys { + uniKeys[key.Column.Name.L] = struct{}{} + } + // Extract every unique key and primary key. + if len(uniKeys) != 0 { + constraints = append(constraints, uniKeys) + } + } + } + return constraints +} + +// checkConstraintIncludePartKey checks that the partitioning key is included in the constraint. +func checkConstraintIncludePartKey(partkeys []string, constraints map[string]struct{}) bool { + for _, pk := range partkeys { + if _, ok := constraints[pk]; !ok { + return false + } + } + return true +} From 9813c714989e67033f70c023bb5f2e29788cf5a0 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 7 Aug 2018 10:34:20 +0800 Subject: [PATCH 07/87] *: support update operation for table partition (#7166) --- executor/write.go | 7 ++- executor/write_test.go | 102 ++++++++++++++++++++++++++++++++++++++ table/tables/partition.go | 50 +++++++++++++++++++ 3 files changed, 158 insertions(+), 1 deletion(-) diff --git a/executor/write.go b/executor/write.go index ab20f07ab0db1..90365f4423c80 100644 --- a/executor/write.go +++ b/executor/write.go @@ -147,7 +147,12 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu skipHandleCheck := false if sc.DupKeyAsWarning { // if the new handle exists. `UPDATE IGNORE` will avoid removing record, and do nothing. - if err = tables.CheckHandleExists(ctx, t, newHandle); err != nil { + if t1, ok := t.(*tables.PartitionedTable); ok { + err = t1.CheckHandleExists(ctx, newHandle, newData) + } else { + err = tables.CheckHandleExists(ctx, t, newHandle) + } + if err != nil { return false, handleChanged, newHandle, 0, errors.Trace(err) } skipHandleCheck = true diff --git a/executor/write_test.go b/executor/write_test.go index d33360d636a4a..f953a00a88209 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -918,6 +918,108 @@ func (s *testSuite) TestUpdate(c *C) { tk.MustExec("update (select * from t) t set c1 = 1111111") } +func (s *testSuite) TestPartitionedTableUpdate(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@session.tidb_enable_table_partition=1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`create table t (id int not null default 1, name varchar(255)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) + + tk.MustExec(`insert INTO t VALUES (1, "hello");`) + tk.CheckExecResult(1, 0) + tk.MustExec(`insert INTO t VALUES (7, "hello");`) + tk.CheckExecResult(1, 0) + + // update non partition column + tk.MustExec(`UPDATE t SET name = "abc" where id > 0;`) + tk.CheckExecResult(2, 0) + r := tk.MustQuery(`SELECT * from t order by id limit 2;`) + r.Check(testkit.Rows("1 abc", "7 abc")) + + // update partition column + tk.MustExec(`update t set id = id + 1`) + tk.CheckExecResult(2, 0) + r = tk.MustQuery(`SELECT * from t order by id limit 2;`) + r.Check(testkit.Rows("2 abc", "8 abc")) + + // update partition column, old and new record locates on different partitions + tk.MustExec(`update t set id = 20 where id = 8`) + tk.CheckExecResult(2, 0) + r = tk.MustQuery(`SELECT * from t order by id limit 2;`) + r.Check(testkit.Rows("2 abc", "20 abc")) + + // table option is auto-increment + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (id int not null auto_increment, name varchar(255), primary key(id)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) + + tk.MustExec("insert into t(name) values ('aa')") + tk.MustExec("update t set id = 8 where name = 'aa'") + tk.MustExec("insert into t(name) values ('bb')") + r = tk.MustQuery("select * from t;") + r.Check(testkit.Rows("8 aa", "9 bb")) + + _, err := tk.Exec("update t set id = null where name = 'aa'") + c.Assert(err, NotNil) + c.Assert(err.Error(), DeepEquals, "[table:1048]Column 'id' cannot be null") + + // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (id int, name int unique) + PARTITION BY RANGE ( name ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) + tk.MustExec("insert t values (1, 1), (2, 2);") + _, err = tk.Exec("update t set name = 1 where id = 2") + c.Assert(err, NotNil) + tk.MustQuery("select * from t").Check(testkit.Rows("1 1", "2 2")) + + // test update ignore for pimary key + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a bigint, primary key (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11))`) + tk.MustExec("insert into t values (5)") + tk.MustExec("insert into t values (7)") + _, err = tk.Exec("update ignore t set a = 5 where a = 7;") + c.Assert(err, IsNil) + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 'PRIMARY'")) + tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) + + // test update ignore for truncate as warning + _, err = tk.Exec("update ignore t set a = 1 where a = (select '2a')") + c.Assert(err, IsNil) + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1265 Data Truncated", "Warning 1265 Data Truncated")) + + // test update ignore for unique key + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a bigint, unique key I_uniq (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11))`) + tk.MustExec("insert into t values (5)") + tk.MustExec("insert into t values (7)") + _, err = tk.Exec("update ignore t set a = 5 where a = 7;") + c.Assert(err, IsNil) + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 'I_uniq'")) + tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) +} + // TestUpdateCastOnlyModifiedValues for issue #4514. func (s *testSuite) TestUpdateCastOnlyModifiedValues(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/table/tables/partition.go b/table/tables/partition.go index 3025508dc70e5..c4fafcd96cc85 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -205,3 +205,53 @@ func (t *PartitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []typ tbl := t.GetPartition(pid) return tbl.RemoveRecord(ctx, h, r) } + +// UpdateRecord implements table.Table UpdateRecord interface. +// `touched` means which columns are really modified, used for secondary indices. +// Length of `oldData` and `newData` equals to length of `t.WritableCols()`. +func (t *PartitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error { + partitionInfo := t.meta.GetPartitionInfo() + from, err := t.locatePartition(ctx, partitionInfo, currData) + if err != nil { + return errors.Trace(err) + } + to, err := t.locatePartition(ctx, partitionInfo, newData) + if err != nil { + return errors.Trace(err) + } + + // The old and new data locate in different partitions. + // Remove record from old partition and add record to new partition. + if from != to { + _, err = t.GetPartition(to).AddRecord(ctx, newData, false) + if err != nil { + return errors.Trace(err) + } + // UpdateRecord should be side effect free, but there're two steps here. + // What would happen if step1 succeed but step2 meets error? It's hard + // to rollback. + // So this special order is chosen: add record first, errors such as + // 'Key Already Exists' will generally happen during step1, errors are + // unlikely to happen in step2. + err = t.GetPartition(from).RemoveRecord(ctx, h, currData) + if err != nil { + log.Error("partition update record error, it may write dirty data to txn:", errors.ErrorStack(err)) + return errors.Trace(err) + } + return nil + } + + tbl := t.GetPartition(to) + return tbl.UpdateRecord(ctx, h, currData, newData, touched) +} + +// CheckHandleExists check whether recordID key exists. if not exists, return nil, +// otherwise return kv.ErrKeyExists error. +func (t *PartitionedTable) CheckHandleExists(ctx sessionctx.Context, handle int64, data []types.Datum) error { + info := t.Meta().GetPartitionInfo() + pid, err := t.locatePartition(ctx, info, data) + if err != nil { + return errors.Trace(err) + } + return CheckHandleExists(ctx, t.GetPartition(pid), handle) +} From 016b92d576917e3a73fa947345205cd338e8abae Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 7 Aug 2018 10:46:47 +0800 Subject: [PATCH 08/87] add timeout for make race (#7283) --- Makefile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index cb09a38c9a0ab..ca63c10cc02c5 100644 --- a/Makefile +++ b/Makefile @@ -149,20 +149,20 @@ race: parserlib go get github.com/coreos/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ - $(GOTEST) -race $(PACKAGES) + $(GOTEST) -timeout 20m -race $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) leak: parserlib go get github.com/coreos/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ - $(GOTEST) -tags leak $(PACKAGES) + $(GOTEST) -tags leak $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) tikv_integration_test: parserlib go get github.com/coreos/gofail @$(GOFAIL_ENABLE) - $(GOTEST) ./store/tikv/. -with-tikv=true + $(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) RACE_FLAG = From 4b7a77e2c1f5318eea5c65404259e8639912596a Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 7 Aug 2018 11:45:49 +0800 Subject: [PATCH 09/87] perfschema: simplify performance schema (#7263) --- mysql/errcode.go | 2 +- mysql/errname.go | 2 +- perfschema/const.go | 140 ++++++++++++------------ perfschema/init.go | 60 +++++------ perfschema/perfschema.go | 12 +-- perfschema/tables.go | 157 +++++++++++++++++++++++++++ perfschema/tables_test.go | 53 +++++++++ perfschema/virtual_tables.go | 100 ----------------- perfschema/virtual_tables_test.go | 71 ------------ table/tables/virtual_tables.go | 174 ------------------------------ 10 files changed, 313 insertions(+), 458 deletions(-) create mode 100644 perfschema/tables.go create mode 100644 perfschema/tables_test.go delete mode 100644 perfschema/virtual_tables.go delete mode 100644 perfschema/virtual_tables_test.go delete mode 100644 table/tables/virtual_tables.go diff --git a/mysql/errcode.go b/mysql/errcode.go index 30813c1693824..be74573c1d6bf 100644 --- a/mysql/errcode.go +++ b/mysql/errcode.go @@ -700,7 +700,7 @@ const ( ErrPathLength = 1680 ErrWarnDeprecatedSyntaxNoReplacement = 1681 ErrWrongNativeTableStructure = 1682 - ErrWrongPerfschemaUsage = 1683 + ErrWrongPerfSchemaUsage = 1683 ErrWarnISSkippedTable = 1684 ErrInsideTransactionPreventsSwitchBinlogDirect = 1685 ErrStoredFunctionPreventsSwitchBinlogDirect = 1686 diff --git a/mysql/errname.go b/mysql/errname.go index f6283b38e5e59..bc26c7c688f47 100644 --- a/mysql/errname.go +++ b/mysql/errname.go @@ -698,7 +698,7 @@ var MySQLErrName = map[uint16]string{ ErrPathLength: "The path specified for %.64s is too long.", ErrWarnDeprecatedSyntaxNoReplacement: "'%s' is deprecated and will be removed in a future release.", ErrWrongNativeTableStructure: "Native table '%-.64s'.'%-.64s' has the wrong structure", - ErrWrongPerfschemaUsage: "Invalid performanceSchema usage.", + ErrWrongPerfSchemaUsage: "Invalid performanceSchema usage.", ErrWarnISSkippedTable: "Table '%s'.'%s' was skipped since its definition is being modified by concurrent DDL statement", ErrInsideTransactionPreventsSwitchBinlogDirect: "Cannot modify @@session.binlogDirectNonTransactionalUpdates inside a transaction", ErrStoredFunctionPreventsSwitchBinlogDirect: "Cannot change the binlog direct flag inside a stored function or trigger", diff --git a/perfschema/const.go b/perfschema/const.go index b5f133567c0f3..9a987530c702b 100644 --- a/perfschema/const.go +++ b/perfschema/const.go @@ -21,61 +21,61 @@ const ( // Definition order same as MySQL's reference manual, so don't bother to // adjust according to alphabetical order. const ( - TableGlobalStatus = "GLOBAL_STATUS" - TableSessionStatus = "SESSION_STATUS" - TableSetupActors = "SETUP_ACTORS" - TableSetupObjects = "SETUP_OBJECTS" - TableSetupInstruments = "SETUP_INSTRUMENTS" - TableSetupConsumers = "SETUP_CONSUMERS" - TableSetupTimers = "SETUP_TIMERS" - TableStmtsCurrent = "EVENTS_STATEMENTS_CURRENT" - TableStmtsHistory = "EVENTS_STATEMENTS_HISTORY" - TableStmtsHistoryLong = "EVENTS_STATEMENTS_HISTORY_LONG" - TablePreparedStmtsInstances = "PREPARED_STATEMENTS_INSTANCES" - TableTransCurrent = "EVENTS_TRANSACTIONS_CURRENT" - TableTransHistory = "EVENTS_TRANSACTIONS_HISTORY" - TableTransHistoryLong = "EVENTS_TRANSACTIONS_HISTORY_LONG" - TableStagesCurrent = "EVENTS_STAGES_CURRENT" - TableStagesHistory = "EVENTS_STAGES_HISTORY" - TableStagesHistoryLong = "EVENTS_STAGES_HISTORY_LONG" + tableGlobalStatus = "GLOBAL_STATUS" + tableSessionStatus = "SESSION_STATUS" + tableSetupActors = "SETUP_ACTORS" + tableSetupObjects = "SETUP_OBJECTS" + tableSetupInstruments = "SETUP_INSTRUMENTS" + tableSetupConsumers = "SETUP_CONSUMERS" + tableSetupTimers = "SETUP_TIMERS" + tableStmtsCurrent = "EVENTS_STATEMENTS_CURRENT" + tableStmtsHistory = "EVENTS_STATEMENTS_HISTORY" + tableStmtsHistoryLong = "EVENTS_STATEMENTS_HISTORY_LONG" + tablePreparedStmtsInstances = "PREPARED_STATEMENTS_INSTANCES" + tableTransCurrent = "EVENTS_TRANSACTIONS_CURRENT" + tableTransHistory = "EVENTS_TRANSACTIONS_HISTORY" + tableTransHistoryLong = "EVENTS_TRANSACTIONS_HISTORY_LONG" + tableStagesCurrent = "EVENTS_STAGES_CURRENT" + tableStagesHistory = "EVENTS_STAGES_HISTORY" + tableStagesHistoryLong = "EVENTS_STAGES_HISTORY_LONG" ) -// PerfSchemaTables is a shortcut to involve all table names. -var PerfSchemaTables = []string{ - TableGlobalStatus, - TableSessionStatus, - TableSetupActors, - TableSetupObjects, - TableSetupInstruments, - TableSetupConsumers, - TableSetupTimers, - TableStmtsCurrent, - TableStmtsHistory, - TableStmtsHistoryLong, - TablePreparedStmtsInstances, - TableTransCurrent, - TableTransHistory, - TableTransHistoryLong, - TableStagesCurrent, - TableStagesHistory, - TableStagesHistoryLong, +// perfSchemaTables is a shortcut to involve all table names. +var perfSchemaTables = []string{ + tableGlobalStatus, + tableSessionStatus, + tableSetupActors, + tableSetupObjects, + tableSetupInstruments, + tableSetupConsumers, + tableSetupTimers, + tableStmtsCurrent, + tableStmtsHistory, + tableStmtsHistoryLong, + tablePreparedStmtsInstances, + tableTransCurrent, + tableTransHistory, + tableTransHistoryLong, + tableStagesCurrent, + tableStagesHistory, + tableStagesHistoryLong, } -// ColumnGlobalStatus contains the column name definitions for table global_status, same as MySQL. +// columnGlobalStatus contains the column name definitions for table global_status, same as MySQL. // // CREATE TABLE performance_schema.global_status( // VARIABLE_NAME VARCHAR(64) not null, // VARIABLE_VALUE VARCHAR(1024)); -var ColumnGlobalStatus = []string{"VARIABLE_NAME", "VARIABLE_VALUE"} +var columnGlobalStatus = []string{"VARIABLE_NAME", "VARIABLE_VALUE"} -// ColumnSessionStatus contains the column name definitions for table session_status, same as MySQL. +// columnSessionStatus contains the column name definitions for table session_status, same as MySQL. // // CREATE TABLE performance_schema.session_status( // VARIABLE_NAME VARCHAR(64) not null, // VARIABLE_VALUE VARCHAR(1024)); -var ColumnSessionStatus = []string{"VARIABLE_NAME", "VARIABLE_VALUE"} +var columnSessionStatus = []string{"VARIABLE_NAME", "VARIABLE_VALUE"} -// ColumnSetupActors contains the column name definitions for table setup_actors, same as MySQL. +// columnSetupActors contains the column name definitions for table setup_actors, same as MySQL. // // CREATE TABLE if not exists performance_schema.setup_actors ( // HOST CHAR(60) NOT NULL DEFAULT '%', @@ -83,9 +83,9 @@ var ColumnSessionStatus = []string{"VARIABLE_NAME", "VARIABLE_VALUE"} // ROLE CHAR(16) NOT NULL DEFAULT '%', // ENABLED ENUM('YES','NO') NOT NULL DEFAULT 'YES', // HISTORY ENUM('YES','NO') NOT NULL DEFAULT 'YES'); -var ColumnSetupActors = []string{"HOST", "USER", "ROLE", "ENABLED", "HISTORY"} +var columnSetupActors = []string{"HOST", "USER", "ROLE", "ENABLED", "HISTORY"} -// ColumnSetupObjects contains the column name definitions for table setup_objects, same as MySQL. +// columnSetupObjects contains the column name definitions for table setup_objects, same as MySQL. // // CREATE TABLE if not exists performance_schema.setup_objects ( // OBJECT_TYPE ENUM('EVENT','FUNCTION','TABLE') NOT NULL DEFAULT 'TABLE', @@ -93,31 +93,31 @@ var ColumnSetupActors = []string{"HOST", "USER", "ROLE", "ENABLED", "HISTORY"} // OBJECT_NAME VARCHAR(64) NOT NULL DEFAULT '%', // ENABLED ENUM('YES','NO') NOT NULL DEFAULT 'YES', // TIMED ENUM('YES','NO') NOT NULL DEFAULT 'YES'); -var ColumnSetupObjects = []string{"OBJECT_TYPE", "OBJECT_SCHEMA", "OBJECT_NAME", "ENABLED", "TIMED"} +var columnSetupObjects = []string{"OBJECT_TYPE", "OBJECT_SCHEMA", "OBJECT_NAME", "ENABLED", "TIMED"} -// ColumnSetupInstruments contains the column name definitions for table setup_instruments, same as MySQL. +// columnSetupInstruments contains the column name definitions for table setup_instruments, same as MySQL. // // CREATE TABLE if not exists performance_schema.setup_instruments ( // NAME VARCHAR(128) NOT NULL, // ENABLED ENUM('YES','NO') NOT NULL, // TIMED ENUM('YES','NO') NOT NULL); -var ColumnSetupInstruments = []string{"NAMED", "ENABLED", "TIMED"} +var columnSetupInstruments = []string{"NAMED", "ENABLED", "TIMED"} -// ColumnSetupConsumers contains the column name definitions for table setup_consumers, same as MySQL. +// columnSetupConsumers contains the column name definitions for table setup_consumers, same as MySQL. // // CREATE TABLE if not exists performance_schema.setup_consumers ( // NAME VARCHAR(64) NOT NULL, // ENABLED ENUM('YES','NO') NOT NULL); -var ColumnSetupConsumers = []string{"NAMED", "ENABLED"} +var columnSetupConsumers = []string{"NAMED", "ENABLED"} -// ColumnSetupTimers contains the column name definitions for table setup_timers, same as MySQL. +// columnSetupTimers contains the column name definitions for table setup_timers, same as MySQL. // // CREATE TABLE if not exists performance_schema.setup_timers ( // NAME VARCHAR(64) NOT NULL, // TIMER_NAME ENUM('NANOSECOND','MICROSECOND','MILLISECOND') NOT NULL); -var ColumnSetupTimers = []string{"NAME", "TIMER_NAME"} +var columnSetupTimers = []string{"NAME", "TIMER_NAME"} -// ColumnStmtsCurrent contains the column name definitions for table events_statements_current, same as MySQL. +// columnStmtsCurrent contains the column name definitions for table events_statements_current, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_statements_current ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -161,7 +161,7 @@ var ColumnSetupTimers = []string{"NAME", "TIMER_NAME"} // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE'), // NESTING_EVENT_LEVEL INT(11)); -var ColumnStmtsCurrent = []string{ +var columnStmtsCurrent = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -205,7 +205,7 @@ var ColumnStmtsCurrent = []string{ "NESTING_EVENT_LEVEL", } -// ColumnStmtsHistory contains the column name definitions for table events_statements_history, same as MySQL. +// columnStmtsHistory contains the column name definitions for table events_statements_history, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_statements_history ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -249,7 +249,7 @@ var ColumnStmtsCurrent = []string{ // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE'), // NESTING_EVENT_LEVEL INT(11)); -var ColumnStmtsHistory = []string{ +var columnStmtsHistory = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -293,7 +293,7 @@ var ColumnStmtsHistory = []string{ "NESTING_EVENT_LEVEL", } -// ColumnStmtsHistoryLong contains the column name definitions for table events_statements_history_long, same as MySQL. +// columnStmtsHistoryLong contains the column name definitions for table events_statements_history_long, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_statements_history_long ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -337,7 +337,7 @@ var ColumnStmtsHistory = []string{ // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE'), // NESTING_EVENT_LEVEL INT(11)); -var ColumnStmtsHistoryLong = []string{ +var columnStmtsHistoryLong = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -381,7 +381,7 @@ var ColumnStmtsHistoryLong = []string{ "NESTING_EVENT_LEVEL", } -// ColumnPreparedStmtsInstances contains the column name definitions for table prepared_statements_instances, same as MySQL. +// columnPreparedStmtsInstances contains the column name definitions for table prepared_statements_instances, same as MySQL. // // CREATE TABLE if not exists performance_schema.prepared_statements_instances ( // OBJECT_INSTANCE_BEGIN BIGINT(20) UNSIGNED NOT NULL, @@ -419,7 +419,7 @@ var ColumnStmtsHistoryLong = []string{ // SUM_SORT_SCAN BIGINT(20) UNSIGNED NOT NULL, // SUM_NO_INDEX_USED BIGINT(20) UNSIGNED NOT NULL, // SUM_NO_GOOD_INDEX_USED BIGINT(20) UNSIGNED NOT NULL); -var ColumnPreparedStmtsInstances = []string{ +var columnPreparedStmtsInstances = []string{ "OBJECT_INSTANCE_BEGIN", "STATEMENT_ID", "STATEMENT_NAME", @@ -457,7 +457,7 @@ var ColumnPreparedStmtsInstances = []string{ "SUM_NO_GOOD_INDEX_USED", } -// ColumnTransCurrent contains the column name definitions for table events_transactions_current, same as MySQL. +// columnTransCurrent contains the column name definitions for table events_transactions_current, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_transactions_current ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -484,7 +484,7 @@ var ColumnPreparedStmtsInstances = []string{ // OBJECT_INSTANCE_BEGIN BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnTransCurrent = []string{ +var columnTransCurrent = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -511,7 +511,7 @@ var ColumnTransCurrent = []string{ "NESTING_EVENT_TYPE", } -// ColumnTransHistory contains the column name definitions for table events_transactions_history, same as MySQL. +// columnTransHistory contains the column name definitions for table events_transactions_history, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_transactions_history ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -538,7 +538,7 @@ var ColumnTransCurrent = []string{ // OBJECT_INSTANCE_BEGIN BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnTransHistory = []string{ +var columnTransHistory = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -565,7 +565,7 @@ var ColumnTransHistory = []string{ "NESTING_EVENT_TYPE", } -// ColumnTransHistoryLong contains the column name definitions for table events_transactions_history_long, same as MySQL. +// columnTransHistoryLong contains the column name definitions for table events_transactions_history_long, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_transactions_history_long ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -592,7 +592,7 @@ var ColumnTransHistory = []string{ // OBJECT_INSTANCE_BEGIN BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnTransHistoryLong = []string{ +var columnTransHistoryLong = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -619,7 +619,7 @@ var ColumnTransHistoryLong = []string{ "NESTING_EVENT_TYPE", } -// ColumnStagesCurrent contains the column name definitions for table events_stages_current, same as MySQL. +// columnStagesCurrent contains the column name definitions for table events_stages_current, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_stages_current ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -634,7 +634,7 @@ var ColumnTransHistoryLong = []string{ // WORK_ESTIMATED BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnStagesCurrent = []string{ +var columnStagesCurrent = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -649,7 +649,7 @@ var ColumnStagesCurrent = []string{ "NESTING_EVENT_TYPE", } -// ColumnStagesHistory contains the column name definitions for table events_stages_history, same as MySQL. +// columnStagesHistory contains the column name definitions for table events_stages_history, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_stages_history ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -664,7 +664,7 @@ var ColumnStagesCurrent = []string{ // WORK_ESTIMATED BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnStagesHistory = []string{ +var columnStagesHistory = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", @@ -679,7 +679,7 @@ var ColumnStagesHistory = []string{ "NESTING_EVENT_TYPE", } -// ColumnStagesHistoryLong contains the column name definitions for table events_stages_history_long, same as MySQL. +// columnStagesHistoryLong contains the column name definitions for table events_stages_history_long, same as MySQL. // // CREATE TABLE if not exists performance_schema.events_stages_history_long ( // THREAD_ID BIGINT(20) UNSIGNED NOT NULL, @@ -694,7 +694,7 @@ var ColumnStagesHistory = []string{ // WORK_ESTIMATED BIGINT(20) UNSIGNED, // NESTING_EVENT_ID BIGINT(20) UNSIGNED, // NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE')); -var ColumnStagesHistoryLong = []string{ +var columnStagesHistoryLong = []string{ "THREAD_ID", "EVENT_ID", "END_EVENT_ID", diff --git a/perfschema/init.go b/perfschema/init.go index 685631cc31325..270afe27f94fd 100644 --- a/perfschema/init.go +++ b/perfschema/init.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/charset" ) @@ -197,7 +196,7 @@ var stagesCurrentCols = []columnInfo{ {mysql.TypeEnum, -1, 0, nil, []string{"TRANSACTION", "STATEMENT", "STAGE"}}, } -func (ps *PerfSchema) buildTables() { +func (ps *perfSchema) buildTables() { tbls := make([]*model.TableInfo, 0, len(ps.tables)) dbID := autoid.GenLocalSchemaID() @@ -207,16 +206,7 @@ func (ps *PerfSchema) buildTables() { for _, c := range meta.Columns { c.ID = autoid.GenLocalSchemaID() } - alloc := autoid.NewMemoryAllocator(dbID) - var tbl table.Table - switch name { - //@TODO in the future, we need to add many VirtualTable, we may need to add new type for these tables. - case TableSessionStatus, TableGlobalStatus: - tbl = createVirtualTable(meta, name) - default: - tbl = tables.MemoryTableFromMeta(alloc, meta) - } - ps.mTables[name] = tbl + ps.mTables[name] = createPerfSchemaTable(meta) } ps.dbInfo = &model.DBInfo{ ID: dbID, @@ -227,7 +217,7 @@ func (ps *PerfSchema) buildTables() { } } -func (ps *PerfSchema) buildModel(tbName string, colNames []string, cols []columnInfo) { +func (ps *perfSchema) buildModel(tbName string, colNames []string, cols []columnInfo) { rcols := make([]*model.ColumnInfo, len(cols)) for i, col := range cols { var ci *model.ColumnInfo @@ -298,7 +288,7 @@ func buildEnumColumnInfo(offset int, name string, elems []string, flag uint, def return colInfo } -func (ps *PerfSchema) initialize() { +func (ps *perfSchema) initialize() { ps.tables = make(map[string]*model.TableInfo) ps.mTables = make(map[string]table.Table, len(ps.tables)) @@ -323,45 +313,45 @@ func (ps *PerfSchema) initialize() { } allColNames := [][]string{ - ColumnGlobalStatus, - ColumnSessionStatus, - ColumnSetupActors, - ColumnSetupObjects, - ColumnSetupInstruments, - ColumnSetupConsumers, - ColumnSetupTimers, - ColumnStmtsCurrent, - ColumnStmtsHistory, - ColumnStmtsHistoryLong, - ColumnPreparedStmtsInstances, - ColumnStmtsCurrent, - ColumnStmtsHistory, - ColumnStmtsHistoryLong, - ColumnStagesCurrent, - ColumnStagesHistory, - ColumnStagesHistoryLong, + columnGlobalStatus, + columnSessionStatus, + columnSetupActors, + columnSetupObjects, + columnSetupInstruments, + columnSetupConsumers, + columnSetupTimers, + columnStmtsCurrent, + columnStmtsHistory, + columnStmtsHistoryLong, + columnPreparedStmtsInstances, + columnStmtsCurrent, + columnStmtsHistory, + columnStmtsHistoryLong, + columnStagesCurrent, + columnStagesHistory, + columnStagesHistoryLong, } // initialize all table, column and result field definitions for i, def := range allColDefs { - ps.buildModel(PerfSchemaTables[i], allColNames[i], def) + ps.buildModel(perfSchemaTables[i], allColNames[i], def) } ps.buildTables() } // GetDBMeta returns the DB info. -func (ps *PerfSchema) GetDBMeta() *model.DBInfo { +func (ps *perfSchema) GetDBMeta() *model.DBInfo { return ps.dbInfo } // GetTable returns the table. -func (ps *PerfSchema) GetTable(name string) (table.Table, bool) { +func (ps *perfSchema) GetTable(name string) (table.Table, bool) { tbl, ok := ps.mTables[name] return tbl, ok } // GetTableMeta returns the table info. -func (ps *PerfSchema) GetTableMeta(name string) (*model.TableInfo, bool) { +func (ps *perfSchema) GetTableMeta(name string) (*model.TableInfo, bool) { tbl, ok := ps.tables[name] return tbl, ok } diff --git a/perfschema/perfschema.go b/perfschema/perfschema.go index bf6994cb6b01a..2e5c1a8106694 100644 --- a/perfschema/perfschema.go +++ b/perfschema/perfschema.go @@ -18,18 +18,18 @@ import ( "github.com/pingcap/tidb/table" ) -// PerfSchema exports for test. -type PerfSchema struct { +// perfSchema is used for tables in performance_schema. +type perfSchema struct { dbInfo *model.DBInfo tables map[string]*model.TableInfo mTables map[string]table.Table // Memory tables for perfSchema } -var handle = NewPerfHandle() +var handle = newPerfHandle() -// NewPerfHandle creates a new perfSchema on store. -func NewPerfHandle() *PerfSchema { - schema := &PerfSchema{} +// newPerfHandle creates a new perfSchema on store. +func newPerfHandle() *perfSchema { + schema := &perfSchema{} schema.initialize() return schema } diff --git a/perfschema/tables.go b/perfschema/tables.go new file mode 100644 index 0000000000000..89bc3ae2cbc7a --- /dev/null +++ b/perfschema/tables.go @@ -0,0 +1,157 @@ +// Copyright 2017 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 perfschema + +import ( + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/types" +) + +// perfSchemaTable stands for the fake table all its data is in the memory. +type perfSchemaTable struct { + meta *model.TableInfo + cols []*table.Column +} + +// createPerfSchemaTable creates all perfSchemaTables +func createPerfSchemaTable(meta *model.TableInfo) *perfSchemaTable { + columns := make([]*table.Column, 0, len(meta.Columns)) + for _, colInfo := range meta.Columns { + col := table.ToColumn(colInfo) + columns = append(columns, col) + } + t := &perfSchemaTable{ + meta: meta, + cols: columns, + } + return t +} + +// IterRecords implements table.Table Type interface. +func (vt *perfSchemaTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, + fn table.RecordIterFunc) error { + if len(startKey) != 0 { + return table.ErrUnsupportedOp + } + return nil +} + +// RowWithCols implements table.Table Type interface. +func (vt *perfSchemaTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { + return nil, table.ErrUnsupportedOp +} + +// Row implements table.Table Type interface. +func (vt *perfSchemaTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { + return nil, table.ErrUnsupportedOp +} + +// Cols implements table.Table Type interface. +func (vt *perfSchemaTable) Cols() []*table.Column { + return vt.cols +} + +// WritableCols implements table.Table Type interface. +func (vt *perfSchemaTable) WritableCols() []*table.Column { + return vt.cols +} + +// Indices implements table.Table Type interface. +func (vt *perfSchemaTable) Indices() []table.Index { + return nil +} + +// WritableIndices implements table.Table Type interface. +func (vt *perfSchemaTable) WritableIndices() []table.Index { + return nil +} + +// DeletableIndices implements table.Table Type interface. +func (vt *perfSchemaTable) DeletableIndices() []table.Index { + return nil +} + +// RecordPrefix implements table.Table Type interface. +func (vt *perfSchemaTable) RecordPrefix() kv.Key { + return nil +} + +// IndexPrefix implements table.Table Type interface. +func (vt *perfSchemaTable) IndexPrefix() kv.Key { + return nil +} + +// FirstKey implements table.Table Type interface. +func (vt *perfSchemaTable) FirstKey() kv.Key { + return nil +} + +// RecordKey implements table.Table Type interface. +func (vt *perfSchemaTable) RecordKey(h int64) kv.Key { + return nil +} + +// AddRecord implements table.Table Type interface. +func (vt *perfSchemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { + return 0, table.ErrUnsupportedOp +} + +// RemoveRecord implements table.Table Type interface. +func (vt *perfSchemaTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { + return table.ErrUnsupportedOp +} + +// UpdateRecord implements table.Table Type interface. +func (vt *perfSchemaTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { + return table.ErrUnsupportedOp +} + +// AllocAutoID implements table.Table Type interface. +func (vt *perfSchemaTable) AllocAutoID(ctx sessionctx.Context) (int64, error) { + return 0, table.ErrUnsupportedOp +} + +// Allocator implements table.Table Type interface. +func (vt *perfSchemaTable) Allocator(ctx sessionctx.Context) autoid.Allocator { + return nil +} + +// RebaseAutoID implements table.Table Type interface. +func (vt *perfSchemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { + return table.ErrUnsupportedOp +} + +// Meta implements table.Table Type interface. +func (vt *perfSchemaTable) Meta() *model.TableInfo { + return vt.meta +} + +// GetID implements table.Table GetID interface. +func (vt *perfSchemaTable) GetID() int64 { + return vt.meta.ID +} + +// Seek implements table.Table Type interface. +func (vt *perfSchemaTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { + return 0, false, table.ErrUnsupportedOp +} + +// Type implements table.Table Type interface. +func (vt *perfSchemaTable) Type() table.Type { + return table.VirtualTable +} diff --git a/perfschema/tables_test.go b/perfschema/tables_test.go new file mode 100644 index 0000000000000..4349e4e77ca2c --- /dev/null +++ b/perfschema/tables_test.go @@ -0,0 +1,53 @@ +// Copyright 2018 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 perfschema_test + +import ( + "testing" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" +) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +var _ = Suite(&testSuite{}) + +type testSuite struct { +} + +func (s *testSuite) TestPerfSchemaTables(c *C) { + testleak.BeforeTest() + defer testleak.AfterTest(c)() + store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() + do, err := session.BootstrapSession(store) + c.Assert(err, IsNil) + defer do.Close() + + tk := testkit.NewTestKit(c, store) + + tk.MustExec("use performance_schema") + tk.MustQuery("select * from global_status where variable_name = 'Ssl_verify_mode'").Check(testkit.Rows()) + tk.MustQuery("select * from session_status where variable_name = 'Ssl_verify_mode'").Check(testkit.Rows()) + tk.MustQuery("select * from setup_actors").Check(testkit.Rows()) + tk.MustQuery("select * from events_stages_history_long").Check(testkit.Rows()) +} diff --git a/perfschema/virtual_tables.go b/perfschema/virtual_tables.go deleted file mode 100644 index 26f24d4298d46..0000000000000 --- a/perfschema/virtual_tables.go +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2017 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 perfschema - -import ( - "fmt" - - "github.com/juju/errors" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/types" - log "github.com/sirupsen/logrus" -) - -// session/global status decided by scope. -type statusDataSource struct { - meta *model.TableInfo - cols []*table.Column - globalScope bool -} - -// GetRows implements the interface of VirtualDataSource. -func (ds *statusDataSource) GetRows(ctx sessionctx.Context) (fullRows [][]types.Datum, - err error) { - sessionVars := ctx.GetSessionVars() - statusVars, err := variable.GetStatusVars(sessionVars) - if err != nil { - return nil, errors.Trace(err) - } - - var rows = make([][]types.Datum, 0) - for status, v := range statusVars { - if ds.globalScope && v.Scope == variable.ScopeSession { - continue - } - - switch v.Value.(type) { - case []interface{}, nil: - v.Value = fmt.Sprintf("%v", v.Value) - } - value, err := types.ToString(v.Value) - if err != nil { - return nil, errors.Trace(err) - } - row := types.MakeDatums(status, value) - rows = append(rows, row) - } - - return rows, nil -} - -// Meta implements the interface of VirtualDataSource. -func (ds *statusDataSource) Meta() *model.TableInfo { - return ds.meta -} - -// Cols implements the interface of VirtualDataSource. -func (ds *statusDataSource) Cols() []*table.Column { - return ds.cols -} - -// CreateVirtualDataSource is only used for test. -func CreateVirtualDataSource(tableName string, meta *model.TableInfo) (tables.VirtualDataSource, error) { - columns := make([]*table.Column, 0, len(meta.Columns)) - for _, colInfo := range meta.Columns { - col := table.ToColumn(colInfo) - columns = append(columns, col) - } - - switch tableName { - case TableSessionStatus: - return &statusDataSource{meta: meta, cols: columns, globalScope: false}, nil - case TableGlobalStatus: - return &statusDataSource{meta: meta, cols: columns, globalScope: true}, nil - default: - return nil, errors.New("can't find table named by " + tableName) - } -} - -func createVirtualTable(meta *model.TableInfo, tableName string) table.Table { - dataSource, err := CreateVirtualDataSource(tableName, meta) - if err != nil { - log.Fatal(err.Error()) - } - return tables.CreateVirtualTable(dataSource) -} diff --git a/perfschema/virtual_tables_test.go b/perfschema/virtual_tables_test.go deleted file mode 100644 index 978aa9068d79b..0000000000000 --- a/perfschema/virtual_tables_test.go +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright 2017 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 perfschema_test - -import ( - "testing" - - "github.com/juju/errors" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/perfschema" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/mock" -) - -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - -var _ = Suite(&testSuite{}) - -type testSuite struct { -} - -func (*testSuite) TestSessionStatus(c *C) { - store, err := mockstore.NewMockTikvStore() - c.Assert(err, IsNil) - - ctx := mock.NewContext() - ctx.Store = store - ps := perfschema.NewPerfHandle() - - testTableName := []string{perfschema.TableSessionStatus, perfschema.TableGlobalStatus} - for _, tableName := range testTableName { - tb, _ := ps.GetTable(tableName) - meta, ok := ps.GetTableMeta(tableName) - c.Assert(tb, NotNil) - c.Assert(ok, IsTrue) - - sessionStatusHandle, _ := perfschema.CreateVirtualDataSource(tableName, meta) - rows, err := sessionStatusHandle.GetRows(ctx) - c.Assert(err, IsNil) - - c.Assert(findSpecialStatus(rows, "Ssl_cipher"), IsNil) - } -} - -func findSpecialStatus(rows [][]types.Datum, name string) error { - err := errors.New("cant find the status " + name) - for _, row := range rows { - statusNames, _ := row[0].ToString() - if statusNames == name { - err = nil - break - } - } - - return err -} diff --git a/table/tables/virtual_tables.go b/table/tables/virtual_tables.go deleted file mode 100644 index 427589a7829df..0000000000000 --- a/table/tables/virtual_tables.go +++ /dev/null @@ -1,174 +0,0 @@ -// Copyright 2017 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 tables - -import ( - "github.com/juju/errors" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/types" -) - -// VirtualDataSource is used to extract data from the struct in memory. -type VirtualDataSource interface { - // GetRows do the actual job - GetRows(ctx sessionctx.Context) (fullRows [][]types.Datum, err error) - // Meta return the meta of table - Meta() *model.TableInfo - // Cols return the cols of table - Cols() []*table.Column -} - -// VirtualTable stands for the fake table all its data is in the memory. -// dataSource: the function to get rows -// @TODO this table is almost the same as the infoschema tables, but we need to use it in performance schema. -// @TODO So we have to move it here, sometimes we need to refactor the infoschema tables to decrease the multiplicity of the codes -type VirtualTable struct { - dataSource VirtualDataSource -} - -// CreateVirtualTable as its name -func CreateVirtualTable(dataSource VirtualDataSource) *VirtualTable { - return &VirtualTable{dataSource: dataSource} -} - -// IterRecords implements table.Table Type interface. -func (vt *VirtualTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, - fn table.RecordIterFunc) error { - if len(startKey) != 0 { - return table.ErrUnsupportedOp - } - rows, err := vt.dataSource.GetRows(ctx) - if err != nil { - return errors.Trace(err) - } - for i, row := range rows { - more, err := fn(int64(i), row, cols) - if err != nil { - return errors.Trace(err) - } - if !more { - break - } - } - return nil -} - -// RowWithCols implements table.Table Type interface. -func (vt *VirtualTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { - return nil, table.ErrUnsupportedOp -} - -// Row implements table.Table Type interface. -func (vt *VirtualTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { - return nil, table.ErrUnsupportedOp -} - -// Cols implements table.Table Type interface. -func (vt *VirtualTable) Cols() []*table.Column { - return vt.dataSource.Cols() -} - -// WritableCols implements table.Table Type interface. -func (vt *VirtualTable) WritableCols() []*table.Column { - return vt.dataSource.Cols() -} - -// Indices implements table.Table Type interface. -func (vt *VirtualTable) Indices() []table.Index { - return nil -} - -// WritableIndices implements table.Table Type interface. -func (vt *VirtualTable) WritableIndices() []table.Index { - return nil -} - -// DeletableIndices implements table.Table Type interface. -func (vt *VirtualTable) DeletableIndices() []table.Index { - return nil -} - -// RecordPrefix implements table.Table Type interface. -func (vt *VirtualTable) RecordPrefix() kv.Key { - return nil -} - -// IndexPrefix implements table.Table Type interface. -func (vt *VirtualTable) IndexPrefix() kv.Key { - return nil -} - -// FirstKey implements table.Table Type interface. -func (vt *VirtualTable) FirstKey() kv.Key { - return nil -} - -// RecordKey implements table.Table Type interface. -func (vt *VirtualTable) RecordKey(h int64) kv.Key { - return nil -} - -// AddRecord implements table.Table Type interface. -func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { - return 0, table.ErrUnsupportedOp -} - -// RemoveRecord implements table.Table Type interface. -func (vt *VirtualTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { - return table.ErrUnsupportedOp -} - -// UpdateRecord implements table.Table Type interface. -func (vt *VirtualTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { - return table.ErrUnsupportedOp -} - -// AllocAutoID implements table.Table Type interface. -func (vt *VirtualTable) AllocAutoID(ctx sessionctx.Context) (int64, error) { - return 0, table.ErrUnsupportedOp -} - -// Allocator implements table.Table Type interface. -func (vt *VirtualTable) Allocator(ctx sessionctx.Context) autoid.Allocator { - return nil -} - -// RebaseAutoID implements table.Table Type interface. -func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { - return table.ErrUnsupportedOp -} - -// Meta implements table.Table Type interface. -func (vt *VirtualTable) Meta() *model.TableInfo { - return vt.dataSource.Meta() -} - -// GetID implements table.Table GetID interface. -func (vt *VirtualTable) GetID() int64 { - return vt.dataSource.Meta().ID -} - -// Seek implements table.Table Type interface. -func (vt *VirtualTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { - return 0, false, table.ErrUnsupportedOp -} - -// Type implements table.Table Type interface. -func (vt *VirtualTable) Type() table.Type { - return table.VirtualTable -} From ae2e3ac0a7f28453a758b18f1cb87e34f77a5890 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 7 Aug 2018 13:54:27 +0800 Subject: [PATCH 10/87] table: remove useless MemoryTable implement (#7292) --- table/tables/memory_tables.go | 281 ----------------------------- table/tables/memory_tables_test.go | 152 ---------------- 2 files changed, 433 deletions(-) delete mode 100644 table/tables/memory_tables.go delete mode 100644 table/tables/memory_tables_test.go diff --git a/table/tables/memory_tables.go b/table/tables/memory_tables.go deleted file mode 100644 index cd213b7d840ba..0000000000000 --- a/table/tables/memory_tables.go +++ /dev/null @@ -1,281 +0,0 @@ -// Copyright 2016 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 tables - -import ( - "sync" - - "github.com/google/btree" - "github.com/juju/errors" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/types" - log "github.com/sirupsen/logrus" -) - -const btreeDegree = 32 - -type itemKey int64 - -type itemPair struct { - handle itemKey - data []types.Datum -} - -func (r *itemPair) Less(item btree.Item) bool { - switch x := item.(type) { - case itemKey: - return r.handle < x - case *itemPair: - return r.handle < x.handle - } - log.Errorf("invalid type %T", item) - return true -} - -func (k itemKey) Less(item btree.Item) bool { - switch x := item.(type) { - case itemKey: - return k < x - case *itemPair: - return k < x.handle - } - log.Errorf("invalid type %T", item) - return true -} - -// MemoryTable implements table.Table interface. -type MemoryTable struct { - ID int64 - Name model.CIStr - Columns []*table.Column - pkHandleCol *table.Column - - recordPrefix kv.Key - alloc autoid.Allocator - meta *model.TableInfo - - tree *btree.BTree - mu sync.RWMutex -} - -// MemoryTableFromMeta creates a Table instance from model.TableInfo. -func MemoryTableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) table.Table { - columns := make([]*table.Column, 0, len(tblInfo.Columns)) - var pkHandleColumn *table.Column - for _, colInfo := range tblInfo.Columns { - col := table.ToColumn(colInfo) - columns = append(columns, col) - if col.IsPKHandleColumn(tblInfo) { - pkHandleColumn = col - } - } - t := newMemoryTable(tblInfo.ID, tblInfo.Name.O, columns, alloc) - t.pkHandleCol = pkHandleColumn - t.meta = tblInfo - return t -} - -// newMemoryTable constructs a MemoryTable instance. -func newMemoryTable(tableID int64, tableName string, cols []*table.Column, alloc autoid.Allocator) *MemoryTable { - name := model.NewCIStr(tableName) - t := &MemoryTable{ - ID: tableID, - Name: name, - alloc: alloc, - Columns: cols, - recordPrefix: tablecodec.GenTableRecordPrefix(tableID), - tree: btree.New(btreeDegree), - } - return t -} - -// Seek seeks the handle -func (t *MemoryTable) Seek(ctx sessionctx.Context, handle int64) (int64, bool, error) { - var found bool - var result int64 - t.mu.RLock() - t.tree.AscendGreaterOrEqual(itemKey(handle), func(item btree.Item) bool { - found = true - result = int64(item.(*itemPair).handle) - return false - }) - t.mu.RUnlock() - return result, found, nil -} - -// Indices implements table.Table Indices interface. -func (t *MemoryTable) Indices() []table.Index { - return nil -} - -// WritableIndices implements table.Table WritableIndices interface. -func (t *MemoryTable) WritableIndices() []table.Index { - return nil -} - -// DeletableIndices implements table.Table DeletableIndices interface. -func (t *MemoryTable) DeletableIndices() []table.Index { - return nil -} - -// Meta implements table.Table Meta interface. -func (t *MemoryTable) Meta() *model.TableInfo { - return t.meta -} - -// GetID implements table.Table GetID interface. -func (t *MemoryTable) GetID() int64 { - return t.meta.ID -} - -// Cols implements table.Table Cols interface. -func (t *MemoryTable) Cols() []*table.Column { - return t.Columns -} - -// WritableCols implements table.Table WritableCols interface. -func (t *MemoryTable) WritableCols() []*table.Column { - return t.Columns -} - -// RecordPrefix implements table.Table RecordPrefix interface. -func (t *MemoryTable) RecordPrefix() kv.Key { - return t.recordPrefix -} - -// IndexPrefix implements table.Table IndexPrefix interface. -func (t *MemoryTable) IndexPrefix() kv.Key { - return nil -} - -// RecordKey implements table.Table RecordKey interface. -func (t *MemoryTable) RecordKey(h int64) kv.Key { - return tablecodec.EncodeRecordKey(t.recordPrefix, h) -} - -// FirstKey implements table.Table FirstKey interface. -func (t *MemoryTable) FirstKey() kv.Key { - return t.RecordKey(0) -} - -// Truncate drops all data in Memory Table. -func (t *MemoryTable) Truncate() { - t.tree = btree.New(btreeDegree) -} - -// UpdateRecord implements table.Table UpdateRecord interface. -func (t *MemoryTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { - t.mu.Lock() - defer t.mu.Unlock() - item := t.tree.Get(itemKey(h)) - if item == nil { - return table.ErrRowNotFound - } - pair := item.(*itemPair) - pair.data = newData - return nil -} - -// AddRecord implements table.Table AddRecord interface. -func (t *MemoryTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { - if t.pkHandleCol != nil { - recordID, err = r[t.pkHandleCol.Offset].ToInt64(ctx.GetSessionVars().StmtCtx) - if err != nil { - return 0, errors.Trace(err) - } - } else { - recordID, err = t.alloc.Alloc(t.ID) - if err != nil { - return 0, errors.Trace(err) - } - } - item := &itemPair{ - handle: itemKey(recordID), - data: r, - } - t.mu.Lock() - defer t.mu.Unlock() - if t.tree.Get(itemKey(recordID)) != nil { - return 0, kv.ErrKeyExists - } - t.tree.ReplaceOrInsert(item) - return -} - -// RowWithCols implements table.Table RowWithCols interface. -func (t *MemoryTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { - t.mu.RLock() - defer t.mu.RUnlock() - item := t.tree.Get(itemKey(h)) - if item == nil { - return nil, table.ErrRowNotFound - } - row := item.(*itemPair).data - v := make([]types.Datum, len(cols)) - for i, col := range cols { - if col == nil { - continue - } - v[i] = row[col.Offset] - } - return v, nil -} - -// Row implements table.Table Row interface. -func (t *MemoryTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { - r, err := t.RowWithCols(nil, h, t.Cols()) - if err != nil { - return nil, errors.Trace(err) - } - return r, nil -} - -// RemoveRecord implements table.Table RemoveRecord interface. -func (t *MemoryTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { - t.mu.Lock() - t.tree.Delete(itemKey(h)) - t.mu.Unlock() - return nil -} - -// AllocAutoID implements table.Table AllocAutoID interface. -func (t *MemoryTable) AllocAutoID(ctx sessionctx.Context) (int64, error) { - return t.alloc.Alloc(t.ID) -} - -// Allocator implements table.Table Allocator interface. -func (t *MemoryTable) Allocator(ctx sessionctx.Context) autoid.Allocator { - return t.alloc -} - -// RebaseAutoID implements table.Table RebaseAutoID interface. -func (t *MemoryTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { - return t.alloc.Rebase(t.ID, newBase, isSetStep) -} - -// IterRecords implements table.Table IterRecords interface. -func (t *MemoryTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, - fn table.RecordIterFunc) error { - return nil -} - -// Type implements table.Table Type interface. -func (t *MemoryTable) Type() table.Type { - return table.MemoryTable -} diff --git a/table/tables/memory_tables_test.go b/table/tables/memory_tables_test.go deleted file mode 100644 index 3e7958d18424c..0000000000000 --- a/table/tables/memory_tables_test.go +++ /dev/null @@ -1,152 +0,0 @@ -// Copyright 2016 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 tables_test - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" -) - -var _ = Suite(&testMemoryTableSuite{}) - -type testMemoryTableSuite struct { - store kv.Storage - dom *domain.Domain - se session.Session - tbl table.Table -} - -func (ts *testMemoryTableSuite) SetUpSuite(c *C) { - testleak.BeforeTest() - store, err := mockstore.NewMockTikvStore() - c.Check(err, IsNil) - ts.store = store - ts.dom, err = session.BootstrapSession(store) - c.Assert(err, IsNil) - ts.se, err = session.CreateSession4Test(ts.store) - c.Assert(err, IsNil) - - // create table - tp1 := types.NewFieldType(mysql.TypeLong) - col1 := &model.ColumnInfo{ - ID: 1, - Name: model.NewCIStr("a"), - Offset: 0, - FieldType: *tp1, - } - tp2 := types.NewFieldType(mysql.TypeVarchar) - tp2.Flen = 255 - col2 := &model.ColumnInfo{ - ID: 2, - Name: model.NewCIStr("b"), - Offset: 1, - FieldType: *tp2, - } - - tblInfo := &model.TableInfo{ - ID: 100, - Name: model.NewCIStr("t"), - Columns: []*model.ColumnInfo{col1, col2}, - PKIsHandle: true, - } - tblInfo.Columns[0].Flag |= mysql.PriKeyFlag - alloc := autoid.NewMemoryAllocator(int64(10)) - ts.tbl = tables.MemoryTableFromMeta(alloc, tblInfo) -} - -func (ts *testMemoryTableSuite) TearDownSuite(c *C) { - ts.dom.Close() - c.Assert(ts.store.Close(), IsNil) - testleak.AfterTest(c) -} - -func (ts *testMemoryTableSuite) TestMemoryBasic(c *C) { - ctx := ts.se.(sessionctx.Context) - tb := ts.tbl - c.Assert(tb.Meta(), NotNil) - c.Assert(tb.Meta().ID, Greater, int64(0)) - c.Assert(tb.Meta().Name.L, Equals, "t") - c.Assert(tb.Indices(), IsNil) - c.Assert(string(tb.FirstKey()), Not(Equals), "") - c.Assert(string(tb.RecordPrefix()), Not(Equals), "") - - // Basic test for MemoryTable - handle, found, err := tb.Seek(nil, 0) - c.Assert(handle, Equals, int64(0)) - c.Assert(found, Equals, false) - c.Assert(err, IsNil) - cols := tb.WritableCols() - c.Assert(cols, NotNil) - - key := tb.IndexPrefix() - c.Assert(key, IsNil) - err = tb.UpdateRecord(nil, 0, nil, nil, nil) - c.Assert(err, NotNil) - alc := tb.Allocator(nil) - c.Assert(alc, NotNil) - err = tb.RebaseAutoID(nil, 0, false) - c.Assert(err, IsNil) - - autoID, err := tb.AllocAutoID(nil) - c.Assert(err, IsNil) - c.Assert(autoID, Greater, int64(0)) - - rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc"), false) - c.Assert(err, IsNil) - row, err := tb.Row(ctx, rid) - c.Assert(err, IsNil) - c.Assert(len(row), Equals, 2) - c.Assert(row[0].GetInt64(), Equals, int64(1)) - - _, err = tb.AddRecord(ctx, types.MakeDatums(1, "aba"), false) - c.Assert(err, NotNil) - _, err = tb.AddRecord(ctx, types.MakeDatums(2, "abc"), false) - c.Assert(err, IsNil) - - err = tb.UpdateRecord(ctx, 1, types.MakeDatums(1, "abc"), types.MakeDatums(3, "abe"), nil) - c.Assert(err, IsNil) - - tb.IterRecords(ctx, tb.FirstKey(), tb.Cols(), func(h int64, data []types.Datum, cols []*table.Column) (bool, error) { - return true, nil - }) - - // RowWithCols test - vals, err := tb.RowWithCols(ctx, rid, tb.Cols()) - c.Assert(err, IsNil) - c.Assert(vals, HasLen, 2) - c.Assert(vals[0].GetInt64(), Equals, int64(3)) - cols = []*table.Column{tb.Cols()[1]} - vals, err = tb.RowWithCols(ctx, rid, cols) - c.Assert(err, IsNil) - c.Assert(vals, HasLen, 1) - c.Assert(vals[0].GetString(), Equals, "abe") - - c.Assert(tb.RemoveRecord(ctx, rid, types.MakeDatums(1, "cba")), IsNil) - _, err = tb.AddRecord(ctx, types.MakeDatums(1, "abc"), false) - c.Assert(err, IsNil) - tb.(*tables.MemoryTable).Truncate() - _, err = tb.Row(ctx, rid) - c.Assert(err, NotNil) -} From 6e33d2e3b7665c597f8d346290ef2ee7b28c4728 Mon Sep 17 00:00:00 2001 From: Du Chuan Date: Tue, 7 Aug 2018 15:07:02 +0800 Subject: [PATCH 11/87] expression: fix behavior for builtin 'LTrim', 'RTrim' and 'Trim' (#7291) --- expression/builtin_string.go | 2 +- expression/builtin_string_test.go | 17 +++++++++++++++++ expression/integration_test.go | 6 ++++++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 22a6bbd28448d..4998aa24885c8 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -1509,7 +1509,7 @@ func (b *builtinUnHexSig) evalString(row chunk.Row) (string, bool, error) { return string(bs), false, nil } -const spaceChars = "\n\t\r " +const spaceChars = " " type trimFunctionClass struct { baseFunctionClass diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 5425996080143..3d067da53a335 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -869,6 +869,9 @@ func (s *testEvaluatorSuite) TestTrim(c *C) { res string }{ {[]interface{}{" bar "}, false, false, "bar"}, + {[]interface{}{"\t bar \n"}, false, false, "\t bar \n"}, + {[]interface{}{"\r bar \t"}, false, false, "\r bar \t"}, + {[]interface{}{" \tbar\n "}, false, false, "\tbar\n"}, {[]interface{}{""}, false, false, ""}, {[]interface{}{nil}, true, false, ""}, {[]interface{}{"xxxbarxxx", "x"}, false, false, "bar"}, @@ -919,6 +922,14 @@ func (s *testEvaluatorSuite) TestLTrim(c *C) { res string }{ {" bar ", false, false, "bar "}, + {"\t bar ", false, false, "\t bar "}, + {" \tbar ", false, false, "\tbar "}, + {"\t bar ", false, false, "\t bar "}, + {" \tbar ", false, false, "\tbar "}, + {"\r bar ", false, false, "\r bar "}, + {" \rbar ", false, false, "\rbar "}, + {"\n bar ", false, false, "\n bar "}, + {" \nbar ", false, false, "\nbar "}, {"bar", false, false, "bar"}, {"", false, false, ""}, {nil, true, false, ""}, @@ -954,6 +965,12 @@ func (s *testEvaluatorSuite) TestRTrim(c *C) { }{ {" bar ", false, false, " bar"}, {"bar", false, false, "bar"}, + {"bar \n", false, false, "bar \n"}, + {"bar\n ", false, false, "bar\n"}, + {"bar \r", false, false, "bar \r"}, + {"bar\r ", false, false, "bar\r"}, + {"bar \t", false, false, "bar \t"}, + {"bar\t ", false, false, "bar\t"}, {"", false, false, ""}, {nil, true, false, ""}, {errors.New("must error"), false, true, ""}, diff --git a/expression/integration_test.go b/expression/integration_test.go index 50462a3eb43b9..6dc31251a5354 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -763,6 +763,10 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { result.Check(testutil.RowsWithSep(",", "bar ,bar,,")) result = tk.MustQuery(`select rtrim(' bar '), rtrim('bar'), rtrim(''), rtrim(null)`) result.Check(testutil.RowsWithSep(",", " bar,bar,,")) + result = tk.MustQuery(`select ltrim("\t bar "), ltrim(" \tbar"), ltrim("\n bar"), ltrim("\r bar")`) + result.Check(testutil.RowsWithSep(",", "\t bar ,\tbar,\n bar,\r bar")) + result = tk.MustQuery(`select rtrim(" bar \t"), rtrim("bar\t "), rtrim("bar \n"), rtrim("bar \r")`) + result.Check(testutil.RowsWithSep(",", " bar \t,bar\t,bar \n,bar \r")) // for reverse tk.MustExec(`DROP TABLE IF EXISTS t;`) @@ -776,6 +780,8 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { // for trim result = tk.MustQuery(`select trim(' bar '), trim(leading 'x' from 'xxxbarxxx'), trim(trailing 'xyz' from 'barxxyz'), trim(both 'x' from 'xxxbarxxx')`) result.Check(testkit.Rows("bar barxxx barx bar")) + result = tk.MustQuery(`select trim('\t bar\n '), trim(' \rbar \t')`) + result.Check(testutil.RowsWithSep(",", "\t bar\n,\rbar \t")) result = tk.MustQuery(`select trim(leading from ' bar'), trim('x' from 'xxxbarxxx'), trim('x' from 'bar'), trim('' from ' bar ')`) result.Check(testutil.RowsWithSep(",", "bar,bar,bar, bar ")) result = tk.MustQuery(`select trim(''), trim('x' from '')`) From f6dbad0f5c3cc42cafdfa00275abbd2197b8376b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 7 Aug 2018 15:38:01 +0800 Subject: [PATCH 12/87] *: move the dirty table modification to table package (#7289) --- executor/delete.go | 1 - executor/insert.go | 3 --- executor/replace.go | 4 ---- executor/union_scan.go | 2 +- executor/write.go | 17 ----------------- session/txn.go | 7 ++++--- table/table.go | 9 +++++++++ table/tables/partition.go | 4 +--- table/tables/tables.go | 10 +++++++++- 9 files changed, 24 insertions(+), 33 deletions(-) diff --git a/executor/delete.go b/executor/delete.go index de2ef6f1090a0..77f37e73c27b8 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -223,7 +223,6 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d if err != nil { return errors.Trace(err) } - ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, t.Meta().ID, h, nil) ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) colSize := make(map[int64]int64) for id, col := range t.Cols() { diff --git a/executor/insert.go b/executor/insert.go index 45c9abf4d607e..c497b7bac2d4e 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -47,9 +47,6 @@ func (e *InsertExec) insertOneRow(row []types.Datum) (int64, error) { if err != nil { return 0, errors.Trace(err) } - if !e.ctx.GetSessionVars().LightningMode { - e.ctx.StmtAddDirtyTableOP(DirtyTableAddRow, e.Table.Meta().ID, h, row) - } e.batchInsertRowCount++ return h, nil } diff --git a/executor/replace.go b/executor/replace.go index fb6b9bf33dae8..1f00648c67293 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -65,7 +65,6 @@ func (e *ReplaceExec) removeRow(handle int64, newRow []types.Datum) (bool, error if err != nil { return false, errors.Trace(err) } - e.ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, e.Table.Meta().ID, handle, nil) e.ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) // Cleanup keys map, because the record was removed. @@ -89,9 +88,6 @@ func (e *ReplaceExec) addRow(row []types.Datum) (int64, error) { if err != nil { return 0, errors.Trace(err) } - if !e.ctx.GetSessionVars().LightningMode { - e.ctx.StmtAddDirtyTableOP(DirtyTableAddRow, e.Table.Meta().ID, h, row) - } return h, nil } diff --git a/executor/union_scan.go b/executor/union_scan.go index b993fef90b53f..5efdd74d80a1f 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -92,7 +92,7 @@ func GetDirtyDB(ctx sessionctx.Context) *DirtyDB { return udb } -// UnionScanExec merges the rows from dirty table and the rows from XAPI request. +// UnionScanExec merges the rows from dirty table and the rows from distsql request. type UnionScanExec struct { baseExecutor diff --git a/executor/write.go b/executor/write.go index 90365f4423c80..3a046c12a35a2 100644 --- a/executor/write.go +++ b/executor/write.go @@ -34,15 +34,6 @@ var ( _ Executor = &LoadDataExec{} ) -const ( - // DirtyTableAddRow is the constant for dirty table operation type. - DirtyTableAddRow = iota - // DirtyTableDeleteRow is the constant for dirty table operation type. - DirtyTableDeleteRow - // DirtyTableTruncate is the constant for dirty table operation type. - DirtyTableTruncate -) - // updateRecord updates the row specified by the handle `h`, from `oldData` to `newData`. // `modified` means which columns are really modified. It's used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. @@ -170,14 +161,6 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu return false, handleChanged, newHandle, 0, errors.Trace(err) } - tid := t.Meta().ID - ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, tid, h, nil) - if handleChanged { - ctx.StmtAddDirtyTableOP(DirtyTableAddRow, tid, newHandle, newData) - } else { - ctx.StmtAddDirtyTableOP(DirtyTableAddRow, tid, h, newData) - } - if onDup { sc.AddAffectedRows(2) } else { diff --git a/session/txn.go b/session/txn.go index 5a2d17a055024..85a64cd77836e 100644 --- a/session/txn.go +++ b/session/txn.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" binlog "github.com/pingcap/tipb/go-binlog" log "github.com/sirupsen/logrus" @@ -218,11 +219,11 @@ func mergeToMutation(m1, m2 *binlog.TableMutation) { func mergeToDirtyDB(dirtyDB *executor.DirtyDB, op dirtyTableOperation) { switch op.kind { - case executor.DirtyTableAddRow: + case table.DirtyTableAddRow: dirtyDB.AddRow(op.tid, op.handle, op.row) - case executor.DirtyTableDeleteRow: + case table.DirtyTableDeleteRow: dirtyDB.DeleteRow(op.tid, op.handle) - case executor.DirtyTableTruncate: + case table.DirtyTableTruncate: dirtyDB.TruncateTable(op.tid) } } diff --git a/table/table.go b/table/table.go index 3bc0eb0bb6d1a..300fa9a83045c 100644 --- a/table/table.go +++ b/table/table.go @@ -39,6 +39,15 @@ const ( MemoryTable ) +const ( + // DirtyTableAddRow is the constant for dirty table operation type. + DirtyTableAddRow = iota + // DirtyTableDeleteRow is the constant for dirty table operation type. + DirtyTableDeleteRow + // DirtyTableTruncate is the constant for dirty table operation type. + DirtyTableTruncate +) + var ( // ErrColumnCantNull is used for inserting null to a not null column. ErrColumnCantNull = terror.ClassTable.New(codeColumnCantNull, mysql.MySQLErrName[mysql.ErrBadNull]) diff --git a/table/tables/partition.go b/table/tables/partition.go index c4fafcd96cc85..747c2f3a5c25c 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -47,12 +47,11 @@ var _ table.PartitionedTable = &PartitionedTable{} // Partition also implements the table.Table interface. type Partition struct { tableCommon - ID int64 } // GetID implements table.Table GetID interface. func (p *Partition) GetID() int64 { - return p.ID + return p.partitionID } // PartitionedTable implements the table.PartitionedTable interface. @@ -77,7 +76,6 @@ func newPartitionedTable(tbl *Table, tblInfo *model.TableInfo) (table.Table, err if err != nil { return nil, errors.Trace(err) } - t.ID = p.ID partitions[p.ID] = &t } diff --git a/table/tables/tables.go b/table/tables/tables.go index 72438abf779b5..bc9a0081f7e87 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -207,7 +207,7 @@ func (t *tableCommon) Meta() *model.TableInfo { // GetID implements table.Table GetID interface. func (t *tableCommon) GetID() int64 { - return t.meta.ID + return t.partitionID } // Cols implements table.Table Cols interface. @@ -326,6 +326,8 @@ func (t *tableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new if err = bs.SaveTo(txn); err != nil { return errors.Trace(err) } + ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.partitionID, h, nil) + ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.partitionID, h, newData) if shouldWriteBinlog(ctx) { if !t.meta.PKIsHandle { binlogColIDs = append(binlogColIDs, model.ExtraHandleID) @@ -478,6 +480,10 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHan return 0, errors.Trace(err) } } + + if !ctx.GetSessionVars().LightningMode { + ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.partitionID, recordID, r) + } if shouldWriteBinlog(ctx) { // For insert, TiDB and Binlog can use same row and schema. binlogRow = row @@ -637,6 +643,8 @@ func (t *tableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Da if err != nil { return errors.Trace(err) } + + ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.partitionID, h, nil) if shouldWriteBinlog(ctx) { cols := t.Cols() colIDs := make([]int64, 0, len(cols)+1) From 61b4ff23213de3da2eb152a83b04ed63328ef2ad Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 7 Aug 2018 16:39:54 +0800 Subject: [PATCH 13/87] ddl: add fsp checking for time/timestamp/datetime column definition (#7294) --- ddl/db_test.go | 22 ++++++++++++++++++++++ ddl/ddl_api.go | 20 ++++++++++++-------- 2 files changed, 34 insertions(+), 8 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index c792ee826060f..65fc9bd81a6a7 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1739,6 +1739,28 @@ func (s *testDBSuite) TestTableDDLWithFloatType(c *C) { s.mustExec(c, "drop table t") } +func (s *testDBSuite) TestTableDDLWithTimeType(c *C) { + s.tk.MustExec("use test") + s.tk.MustExec("drop table if exists t") + s.testErrorCode(c, "create table t (a time(7))", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "create table t (a datetime(7))", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "create table t (a timestamp(7))", tmysql.ErrTooBigPrecision) + _, err := s.tk.Exec("create table t (a time(-1))") + c.Assert(err, NotNil) + s.mustExec(c, "create table t (a datetime)") + s.testErrorCode(c, "alter table t add column b time(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t add column b datetime(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t add column b timestamp(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t modify column a time(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t modify column a datetime(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t modify column a timestamp(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t change column a aa time(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t change column a aa datetime(7)", tmysql.ErrTooBigPrecision) + s.testErrorCode(c, "alter table t change column a aa timestamp(7)", tmysql.ErrTooBigPrecision) + s.mustExec(c, "alter table t change column a aa timestamp(0)") + s.mustExec(c, "drop table t") +} + func (s *testDBSuite) TestTruncateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5ae069cb0232b..613d4a42a6f49 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -597,23 +597,27 @@ func checkTooManyColumns(colDefs []*ast.ColumnDef) error { return nil } -// checkPointTypeColumns checks multiple decimal/float/double columns. -func checkPointTypeColumns(colDefs []*ast.ColumnDef) error { +// checkColumnsAttributes checks attributes for multiple columns. +func checkColumnsAttributes(colDefs []*ast.ColumnDef) error { for _, colDef := range colDefs { - if err := checkPointTypeColumn(colDef.Name.OrigColName(), colDef.Tp); err != nil { + if err := checkColumnAttributes(colDef.Name.OrigColName(), colDef.Tp); err != nil { return errors.Trace(err) } } return nil } -// checkPointTypeColumn checks a decimal/float/double column. -func checkPointTypeColumn(colName string, tp *types.FieldType) error { +// checkColumnAttributes check attributes for single column. +func checkColumnAttributes(colName string, tp *types.FieldType) error { switch tp.Tp { case mysql.TypeNewDecimal, mysql.TypeDouble, mysql.TypeFloat: if tp.Flen < tp.Decimal { return types.ErrMBiggerThanD.GenByArgs(colName) } + case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: + if tp.Decimal != types.UnspecifiedFsp && (tp.Decimal < types.MinFsp || tp.Decimal > types.MaxFsp) { + return types.ErrTooBigPrecision.GenByArgs(tp.Decimal, colName, types.MaxFsp) + } } return nil } @@ -871,7 +875,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e return errors.Trace(err) } - if err = checkPointTypeColumns(colDefs); err != nil { + if err = checkColumnsAttributes(colDefs); err != nil { return errors.Trace(err) } @@ -1193,7 +1197,7 @@ func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTab } colName := specNewColumn.Name.Name.O - if err = checkPointTypeColumn(colName, specNewColumn.Tp); err != nil { + if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil { return errors.Trace(err) } @@ -1569,7 +1573,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(errUnsupportedModifyColumn) } - if err = checkPointTypeColumn(specNewColumn.Name.OrigColName(), specNewColumn.Tp); err != nil { + if err = checkColumnAttributes(specNewColumn.Name.OrigColName(), specNewColumn.Tp); err != nil { return nil, errors.Trace(err) } From 3a3845fb24a971f57fd0abab999799a9ff3562c8 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 7 Aug 2018 18:23:41 +0800 Subject: [PATCH 14/87] *: support admin show ddl (#7274) --- ddl/stat.go | 37 ++++++++++++++++++++----------------- executor/executor.go | 12 ++++++++---- meta/meta.go | 2 +- plan/planbuilder.go | 2 +- util/admin/admin.go | 22 +++++++++++++++++----- util/admin/admin_test.go | 18 +++++++++++++++++- 6 files changed, 64 insertions(+), 29 deletions(-) diff --git a/ddl/stat.go b/ddl/stat.go index 7b5ec8f0387e6..63917f0f88e16 100644 --- a/ddl/stat.go +++ b/ddl/stat.go @@ -65,23 +65,26 @@ func (d *ddl) Stats(vars *variable.SessionVars) (map[string]interface{}, error) m[ddlSchemaVersion] = ddlInfo.SchemaVer // TODO: Get the owner information. - if ddlInfo.Job != nil { - m[ddlJobID] = ddlInfo.Job.ID - m[ddlJobAction] = ddlInfo.Job.Type.String() - m[ddlJobStartTS] = ddlInfo.Job.StartTS / 1e9 // unit: second - m[ddlJobState] = ddlInfo.Job.State.String() - m[ddlJobRows] = ddlInfo.Job.RowCount - if ddlInfo.Job.Error == nil { - m[ddlJobError] = "" - } else { - m[ddlJobError] = ddlInfo.Job.Error.Error() - } - m[ddlJobSchemaState] = ddlInfo.Job.SchemaState.String() - m[ddlJobSchemaID] = ddlInfo.Job.SchemaID - m[ddlJobTableID] = ddlInfo.Job.TableID - m[ddlJobSnapshotVer] = ddlInfo.Job.SnapshotVer - m[ddlJobReorgHandle] = ddlInfo.ReorgHandle - m[ddlJobArgs] = ddlInfo.Job.Args + if len(ddlInfo.Jobs) == 0 { + return m, nil + } + // TODO: Add all job infromation if needed. + job := ddlInfo.Jobs[0] + m[ddlJobID] = job.ID + m[ddlJobAction] = job.Type.String() + m[ddlJobStartTS] = job.StartTS / 1e9 // unit: second + m[ddlJobState] = job.State.String() + m[ddlJobRows] = job.RowCount + if job.Error == nil { + m[ddlJobError] = "" + } else { + m[ddlJobError] = job.Error.Error() } + m[ddlJobSchemaState] = job.SchemaState.String() + m[ddlJobSchemaID] = job.SchemaID + m[ddlJobTableID] = job.TableID + m[ddlJobSnapshotVer] = job.SnapshotVer + m[ddlJobReorgHandle] = ddlInfo.ReorgHandle + m[ddlJobArgs] = job.Args return m, nil } diff --git a/executor/executor.go b/executor/executor.go index 349b0cbd085b1..38454ac9ab077 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -199,13 +199,17 @@ func (e *ShowDDLExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } - ddlJob := "" - if e.ddlInfo.Job != nil { - ddlJob = e.ddlInfo.Job.String() + ddlJobs := "" + l := len(e.ddlInfo.Jobs) + for i, job := range e.ddlInfo.Jobs { + ddlJobs += job.String() + if i != l-1 { + ddlJobs += "\n" + } } chk.AppendInt64(0, e.ddlInfo.SchemaVer) chk.AppendString(1, e.ddlOwnerID) - chk.AppendString(2, ddlJob) + chk.AppendString(2, ddlJobs) chk.AppendString(3, e.selfID) e.done = true return nil diff --git a/meta/meta.go b/meta/meta.go index f662dbcc83ae5..0f9f7a80e3622 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -463,7 +463,7 @@ var ( type JobListKeyType []byte var ( - // DefaultJobListKey keeps all actions of DDL jobs. + // DefaultJobListKey keeps all actions of DDL jobs except "add index". DefaultJobListKey JobListKeyType = mDDLJobListKey // AddIndexJobListKey only keeps the action of adding index. AddIndexJobListKey JobListKeyType = mDDLJobAddIdxList diff --git a/plan/planbuilder.go b/plan/planbuilder.go index 30cc7e8d2524e..2f01bea731a72 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -633,7 +633,7 @@ func buildShowDDLFields() *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, 4)...) schema.Append(buildColumn("", "SCHEMA_VER", mysql.TypeLonglong, 4)) schema.Append(buildColumn("", "OWNER", mysql.TypeVarchar, 64)) - schema.Append(buildColumn("", "JOB", mysql.TypeVarchar, 128)) + schema.Append(buildColumn("", "RUNNING_JOBS", mysql.TypeVarchar, 256)) schema.Append(buildColumn("", "SELF_ID", mysql.TypeVarchar, 64)) return schema diff --git a/util/admin/admin.go b/util/admin/admin.go index efc2a60ebf1df..5a0a3f64a08ca 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -39,8 +39,8 @@ import ( // DDLInfo is for DDL information. type DDLInfo struct { SchemaVer int64 - ReorgHandle int64 // it's only used for DDL information. - Job *model.Job + ReorgHandle int64 // It's only used for DDL information. + Jobs []*model.Job // It's the currently running jobs. } // GetDDLInfo returns DDL information. @@ -49,19 +49,31 @@ func GetDDLInfo(txn kv.Transaction) (*DDLInfo, error) { info := &DDLInfo{} t := meta.NewMeta(txn) - info.Job, err = t.GetDDLJobByIdx(0) + info.Jobs = make([]*model.Job, 0, 2) + job, err := t.GetDDLJobByIdx(0) if err != nil { return nil, errors.Trace(err) } + if job != nil { + info.Jobs = append(info.Jobs, job) + } + addIdxJob, err := t.GetDDLJobByIdx(0, meta.AddIndexJobListKey) + if err != nil { + return nil, errors.Trace(err) + } + if addIdxJob != nil { + info.Jobs = append(info.Jobs, addIdxJob) + } + info.SchemaVer, err = t.GetSchemaVersion() if err != nil { return nil, errors.Trace(err) } - if info.Job == nil { + if addIdxJob == nil { return info, nil } - info.ReorgHandle, _, _, err = t.GetDDLReorgHandle(info.Job) + info.ReorgHandle, _, _, err = t.GetDDLReorgHandle(addIdxJob) if err != nil { return nil, errors.Trace(err) } diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 7dc87d33db35f..ed58353ac894b 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -80,11 +80,27 @@ func (s *testSuite) TestGetDDLInfo(c *C) { Type: model.ActionCreateSchema, RowCount: 0, } + job1 := &model.Job{ + SchemaID: dbInfo2.ID, + Type: model.ActionAddIndex, + RowCount: 0, + } err = t.EnQueueDDLJob(job) c.Assert(err, IsNil) info, err := GetDDLInfo(txn) c.Assert(err, IsNil) - c.Assert(info.Job, DeepEquals, job) + c.Assert(info.Jobs, HasLen, 1) + c.Assert(info.Jobs[0], DeepEquals, job) + c.Assert(info.ReorgHandle, Equals, int64(0)) + // Two jobs. + t = meta.NewMeta(txn, meta.AddIndexJobListKey) + err = t.EnQueueDDLJob(job1) + c.Assert(err, IsNil) + info, err = GetDDLInfo(txn) + c.Assert(err, IsNil) + c.Assert(info.Jobs, HasLen, 2) + c.Assert(info.Jobs[0], DeepEquals, job) + c.Assert(info.Jobs[1], DeepEquals, job1) c.Assert(info.ReorgHandle, Equals, int64(0)) err = txn.Rollback() c.Assert(err, IsNil) From 2babfd9df7ecc3ce34694a0a6b2e49c521565283 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Wed, 8 Aug 2018 14:22:36 +0800 Subject: [PATCH 15/87] plan: make `USE INDEX(`PRIMARY`)` works on the integer primary key (#7298) --- plan/planbuilder.go | 22 +++++++++++++++++----- plan/planbuilder_test.go | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 49 insertions(+), 5 deletions(-) diff --git a/plan/planbuilder.go b/plan/planbuilder.go index 2f01bea731a72..c202c151f2873 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -267,15 +267,27 @@ func (b *planBuilder) detectSelectAgg(sel *ast.SelectStmt) bool { return false } -func getPathByIndexName(paths []*accessPath, idxName model.CIStr) *accessPath { +func getPathByIndexName(paths []*accessPath, idxName model.CIStr, tblInfo *model.TableInfo) *accessPath { + var tablePath *accessPath for _, path := range paths { + if path.isTablePath { + tablePath = path + continue + } if path.index.Name.L == idxName.L { return path } } + if isPrimaryIndexHint(idxName) && tblInfo.PKIsHandle { + return tablePath + } return nil } +func isPrimaryIndexHint(indexName model.CIStr) bool { + return indexName.L == "primary" +} + func getPossibleAccessPaths(indexHints []*ast.IndexHint, tblInfo *model.TableInfo) ([]*accessPath, error) { publicPaths := make([]*accessPath, 0, len(tblInfo.Indices)+1) publicPaths = append(publicPaths, &accessPath{isTablePath: true}) @@ -295,7 +307,7 @@ func getPossibleAccessPaths(indexHints []*ast.IndexHint, tblInfo *model.TableInf hasScanHint = true for _, idxName := range hint.IndexNames { - path := getPathByIndexName(publicPaths[1:], idxName) + path := getPathByIndexName(publicPaths, idxName, tblInfo) if path == nil { return nil, ErrKeyDoesNotExist.GenByArgs(idxName, tblInfo.Name) } @@ -316,7 +328,7 @@ func getPossibleAccessPaths(indexHints []*ast.IndexHint, tblInfo *model.TableInf available = publicPaths } - available = removeIgnoredPaths(available, ignored) + available = removeIgnoredPaths(available, ignored, tblInfo) // If we have got "FORCE" or "USE" index hint but got no available index, // we have to use table scan. @@ -326,13 +338,13 @@ func getPossibleAccessPaths(indexHints []*ast.IndexHint, tblInfo *model.TableInf return available, nil } -func removeIgnoredPaths(paths, ignoredPaths []*accessPath) []*accessPath { +func removeIgnoredPaths(paths, ignoredPaths []*accessPath, tblInfo *model.TableInfo) []*accessPath { if len(ignoredPaths) == 0 { return paths } remainedPaths := make([]*accessPath, 0, len(paths)) for _, path := range paths { - if path.isTablePath || getPathByIndexName(ignoredPaths, path.index.Name) == nil { + if path.isTablePath || getPathByIndexName(ignoredPaths, path.index.Name, tblInfo) == nil { remainedPaths = append(remainedPaths, path) } } diff --git a/plan/planbuilder_test.go b/plan/planbuilder_test.go index e5cfae3d2aac6..4e26a9b701f8f 100644 --- a/plan/planbuilder_test.go +++ b/plan/planbuilder_test.go @@ -16,6 +16,7 @@ package plan import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/model" ) var _ = Suite(&testPlanBuilderSuite{}) @@ -56,3 +57,34 @@ func (s *testPlanBuilderSuite) TestShow(c *C) { } } } + +func (s *testPlanBuilderSuite) TestGetPathByIndexName(c *C) { + tblInfo := &model.TableInfo{ + Indices: make([]*model.IndexInfo, 0), + PKIsHandle: true, + } + + accessPath := []*accessPath{ + {isTablePath: true}, + {index: &model.IndexInfo{Name: model.NewCIStr("idx")}}, + } + + path := getPathByIndexName(accessPath, model.NewCIStr("idx"), tblInfo) + c.Assert(path, NotNil) + c.Assert(path, Equals, accessPath[1]) + + path = getPathByIndexName(accessPath, model.NewCIStr("primary"), tblInfo) + c.Assert(path, NotNil) + c.Assert(path, Equals, accessPath[0]) + + path = getPathByIndexName(accessPath, model.NewCIStr("not exists"), tblInfo) + c.Assert(path, IsNil) + + tblInfo = &model.TableInfo{ + Indices: make([]*model.IndexInfo, 0), + PKIsHandle: false, + } + + path = getPathByIndexName(accessPath, model.NewCIStr("primary"), tblInfo) + c.Assert(path, IsNil) +} From b39b5f579f85985a895b5025121b3f203e0f97c9 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Wed, 8 Aug 2018 15:16:24 +0800 Subject: [PATCH 16/87] executor: refactor `joinResultGenerator` to handle the unmatched outer records (#7288) --- executor/index_lookup_join.go | 22 ++- executor/join.go | 44 ++--- executor/join_result_generators.go | 277 ++++++++++++++++------------- executor/join_test.go | 13 ++ executor/merge_join.go | 14 +- 5 files changed, 216 insertions(+), 154 deletions(-) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 29024d1c2e622..a68d69aa66ec4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -90,8 +90,9 @@ type lookUpJoinTask struct { lookupMap *mvmap.MVMap matchedInners []chunk.Row - doneCh chan error - cursor int + doneCh chan error + cursor int + hasMatch bool memTracker *memory.Tracker // track memory usage. } @@ -205,16 +206,19 @@ func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { } outerRow := task.outerResult.GetRow(task.cursor) - if e.innerIter.Len() == 0 { - err = e.resultGenerator.emit(outerRow, nil, chk) - } else if e.innerIter.Current() != e.innerIter.End() { - err = e.resultGenerator.emit(outerRow, e.innerIter, chk) - } - if err != nil { - return errors.Trace(err) + if e.innerIter.Current() != e.innerIter.End() { + matched, err := e.resultGenerator.tryToMatch(outerRow, e.innerIter, chk) + if err != nil { + return errors.Trace(err) + } + task.hasMatch = task.hasMatch || matched } if e.innerIter.Current() == e.innerIter.End() { + if !task.hasMatch { + e.resultGenerator.onMissMatch(outerRow, chk) + } task.cursor++ + task.hasMatch = false } if chk.NumRows() == e.maxChunkSize { return nil diff --git a/executor/join.go b/executor/join.go index 9b10a1aefe61f..f7e8eba8c8e57 100644 --- a/executor/join.go +++ b/executor/join.go @@ -385,20 +385,14 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R return false, joinResult } if hasNull { - err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - } - return err == nil, joinResult + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + return true, joinResult } e.hashTableValBufs[workerID] = e.hashTable.Get(joinKey, e.hashTableValBufs[workerID][:0]) innerPtrs := e.hashTableValBufs[workerID] if len(innerPtrs) == 0 { - err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - } - return err == nil, joinResult + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + return true, joinResult } innerRows := make([]chunk.Row, 0, len(innerPtrs)) for _, b := range innerPtrs { @@ -407,12 +401,15 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R innerRows = append(innerRows, matchedInner) } iter := chunk.NewIterator4Slice(innerRows) + hasMatch := false for iter.Begin(); iter.Current() != iter.End(); { - err = e.resultGenerators[workerID].emit(outerRow, iter, joinResult.chk) + matched, err := e.resultGenerators[workerID].tryToMatch(outerRow, iter, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) return false, joinResult } + hasMatch = hasMatch || matched + if joinResult.chk.NumRows() == e.maxChunkSize { ok := true e.joinResultCh <- joinResult @@ -422,6 +419,9 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R } } } + if !hasMatch { + e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + } return true, joinResult } @@ -448,11 +448,7 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu } for i := range selected { if !selected[i] { // process unmatched outer rows - err = e.resultGenerators[workerID].emit(outerChk.GetRow(i), nil, joinResult.chk) - if err != nil { - joinResult.err = errors.Trace(err) - return false, joinResult - } + e.resultGenerators[workerID].onMissMatch(outerChk.GetRow(i), joinResult.chk) } else { // process matched outer rows ok, joinResult = e.joinMatchedOuterRow2Chunk(workerID, outerChk.GetRow(i), joinResult) if !ok { @@ -578,6 +574,7 @@ type NestedLoopApplyExec struct { innerSelected []bool innerIter chunk.Iterator outerRow *chunk.Row + hasMatch bool memTracker *memory.Tracker // track memory usage. } @@ -635,9 +632,9 @@ func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *ch if selected { return &outerRow, nil } else if e.outer { - err := e.resultGenerator.emit(outerRow, nil, chk) - if err != nil || chk.NumRows() == e.maxChunkSize { - return nil, errors.Trace(err) + e.resultGenerator.onMissMatch(outerRow, chk) + if chk.NumRows() == e.maxChunkSize { + return nil, nil } } } @@ -678,10 +675,15 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e chk.Reset() for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { + if e.outerRow != nil && !e.hasMatch { + e.resultGenerator.onMissMatch(*e.outerRow, chk) + } e.outerRow, err = e.fetchSelectedOuterRow(ctx, chk) if e.outerRow == nil || err != nil { return errors.Trace(err) } + e.hasMatch = false + for _, col := range e.outerSchema { *col.Data = e.outerRow.GetDatum(col.Index, col.RetType) } @@ -693,7 +695,9 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e e.innerIter.Begin() } - err = e.resultGenerator.emit(*e.outerRow, e.innerIter, chk) + matched, err := e.resultGenerator.tryToMatch(*e.outerRow, e.innerIter, chk) + e.hasMatch = e.hasMatch || matched + if err != nil || chk.NumRows() == e.maxChunkSize { return errors.Trace(err) } diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 759561ee7ecd1..9098502bc2825 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -32,21 +32,51 @@ var ( _ joinResultGenerator = &innerJoinResultGenerator{} ) -// joinResultGenerator is used to generate join results according the join type, see every implementor for detailed information. +// joinResultGenerator is used to generate join results according to the join +// type. A typical instruction flow is: +// +// hasMatch := false +// for innerIter.Current() != innerIter.End() { +// matched, err := g.tryToMatch(outer, innerIter, chk) +// // handle err +// hasMatch = hasMatch || matched +// } +// if !hasMatch { +// g.onMissMatch(outer) +// } +// +// NOTE: This interface is **not** thread-safe. type joinResultGenerator interface { - // emit tries to join an outer row with a batch of inner rows. - // When inners == nil or inners.Len() == 0, it means that the outer row can not be joined with any inner row: - // 1. SemiJoin: unmatched outer row is ignored. - // 2. AntiSemiJoin: unmatched outer row is appended to the result buffer. - // 3. LeftOuterSemiJoin: unmatched outer row is appended with 0 and appended to the result buffer. - // 4. AntiLeftOuterSemiJoin: unmatched outer row is appended with 1 and appended to the result buffer. - // 5. LeftOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. - // 6. RightOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. - // 7. InnerJoin: unmatched outer row is ignored. - // When inners.Len != 0 but all the joined rows are filtered, this means that the outer row is unmatched and the above action is tacked as well. - // Otherwise, the outer row is matched and some joined rows is appended to the `chk`. - // The size of `chk` is MaxChunkSize at most. - emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error + // tryToMatch tries to join an outer row with a batch of inner rows. When + // 'inners.Len != 0' but all the joined rows are filtered, the outer row is + // considered unmatched. Otherwise, the outer row is matched and some joined + // rows are appended to `chk`. The size of `chk` is limited to MaxChunkSize. + // + // NOTE: Callers need to call this function multiple times to consume all + // the inner rows for an outer row, and dicide whether the outer row can be + // matched with at lease one inner row. + tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) + + // onMissMatch operates on the unmatched outer row according to the join + // type. An outer row can be considered miss matched if: + // 1. it can not pass the filter on the outer table side. + // 2. there is no inner row with the same join key. + // 3. all the joined rows can not pass the filter on the join result. + // + // On these conditions, the caller calls this function to handle the + // unmatched outer rows according to the current join type: + // 1. 'SemiJoin': ignores the unmatched outer row. + // 2. 'AntiSemiJoin': appends the unmatched outer row to the result buffer. + // 3. 'LeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 4. 'AntiLeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 5. 'LeftOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. + // 6. 'RightOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. + // 7. 'InnerJoin': ignores the unmatched outer row. + onMissMatch(outer chunk.Row, chk *chunk.Chunk) } func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, @@ -89,8 +119,6 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, panic("unsupported join type in func newJoinResultGenerator()") } -// baseJoinResultGenerator is not thread-safe, -// so we should build individual generator for every join goroutine. type baseJoinResultGenerator struct { ctx sessionctx.Context conditions []expression.Expression @@ -133,15 +161,15 @@ type semiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *semiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - return nil +func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil } - defer inners.ReachEnd() + if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) - return nil + inners.ReachEnd() + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { @@ -151,31 +179,36 @@ func (outputer *semiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iter } else { outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - selected, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) + + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } - if selected { - chk.AppendRow(outer) - return nil + if matched { + chk.AppendPartialRow(0, outer) + inners.ReachEnd() + return true, nil } } - return nil + return false, nil +} + +func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { } type antiSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - chk.AppendRow(outer) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil } - defer inners.ReachEnd() + if len(outputer.conditions) == 0 { - return nil + inners.ReachEnd() + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { @@ -186,183 +219,186 @@ func (outputer *antiSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk. outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } if matched { - return nil + inners.ReachEnd() + return true, nil } } + return false, nil +} + +func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendRow(outer) - return nil } type leftOuterSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil } - defer inners.ReachEnd() if len(outputer.conditions) == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - return nil + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) + + matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } if matched { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - return nil + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } } + return false, nil +} + +func (outputer *leftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 1) +} + +func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) - return nil } type antiLeftOuterSemiJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil } - defer inners.ReachEnd() - // outer row can be joined with an inner row. if len(outputer.conditions) == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - return nil + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) + if err != nil { - return errors.Trace(err) + return false, errors.Trace(err) } - // outer row can be joined with an inner row. if matched { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) - return nil + outputer.onMatch(outer, chk) + inners.ReachEnd() + return true, nil } } + return false, nil +} - // outer row can not be joined with any inner row. +func (outputer *antiLeftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 0) +} + +func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) - return nil } type leftOuterJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } + outputer.chk.Reset() chkForJoin := outputer.chk if len(outputer.conditions) == 0 { chkForJoin = chk } + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk matched, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() - if !matched { - // outer row can not be joined with any inner row. - chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) - } - return nil + return matched, errors.Trace(err) +} + +func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendPartialRow(outer.Len(), outputer.defaultInner) } type rightOuterJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - // outer row can not be joined with any inner row. - if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } + outputer.chk.Reset() chkForJoin := outputer.chk if len(outputer.conditions) == 0 { chkForJoin = chk } + numToAppend := outputer.maxChunkSize - chk.NumRows() for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk matched, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() - // outer row can not be joined with any inner row. - if !matched { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) - } - return nil + return matched, errors.Trace(err) +} + +func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outputer.defaultInner) + chk.AppendPartialRow(outputer.defaultInner.Len(), outer) } type innerJoinResultGenerator struct { baseJoinResultGenerator } -// emit implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { - if inners == nil || inners.Len() == 0 { - return nil +// tryToMatch implements joinResultGenerator interface. +func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil } outputer.chk.Reset() chkForJoin := outputer.chk @@ -378,14 +414,13 @@ func (outputer *innerJoinResultGenerator) emit(outer chunk.Row, inners chunk.Ite } } if len(outputer.conditions) == 0 { - return nil + return true, nil } + // reach here, chkForJoin is outputer.chk - _, err := outputer.filter(chkForJoin, chk) - if err != nil { - return errors.Trace(err) - } - chkForJoin.Reset() + matched, err := outputer.filter(chkForJoin, chk) + return matched, errors.Trace(err) +} - return nil +func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { } diff --git a/executor/join_test.go b/executor/join_test.go index 79b068246c9ea..c8a9b8cad5c58 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -851,6 +851,19 @@ func (s *testSuite) TestIndexLookupJoin(c *C) { `1.01`, `2.02`, )) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a bigint, b bigint, unique key idx1(a, b));`) + tk.MustExec(`insert into t values(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6);`) + tk.MustExec(`set @@tidb_max_chunk_size = 2;`) + tk.MustQuery(`select /*+ TIDB_INLJ(t1) */ * from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b + 4;`).Check(testkit.Rows( + `1 1 `, + `1 2 `, + `1 3 `, + `1 4 `, + `1 5 1 1`, + `1 6 1 2`, + )) } func (s *testSuite) TestMergejoinOrder(c *C) { diff --git a/executor/merge_join.go b/executor/merge_join.go index d2373603e2924..1d23b19ef2274 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -58,8 +58,9 @@ type mergeJoinOuterTable struct { chk *chunk.Chunk selected []bool - iter *chunk.Iterator4Chunk - row chunk.Row + iter *chunk.Iterator4Chunk + row chunk.Row + hasMatch bool } // mergeJoinInnerTable represents the inner table of merge join. @@ -298,12 +299,13 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM } if cmpResult < 0 { - err = e.resultGenerator.emit(e.outerTable.row, nil, chk) + e.resultGenerator.onMissMatch(e.outerTable.row, chk) if err != nil { return false, errors.Trace(err) } e.outerTable.row = e.outerTable.iter.Next() + e.outerTable.hasMatch = false if chk.NumRows() == e.maxChunkSize { return true, nil @@ -311,12 +313,16 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM continue } - err = e.resultGenerator.emit(e.outerTable.row, e.innerIter4Row, chk) + matched, err := e.resultGenerator.tryToMatch(e.outerTable.row, e.innerIter4Row, chk) if err != nil { return false, errors.Trace(err) } + e.outerTable.hasMatch = e.outerTable.hasMatch || matched if e.innerIter4Row.Current() == e.innerIter4Row.End() { + if !e.outerTable.hasMatch { + e.resultGenerator.onMissMatch(e.outerTable.row, chk) + } e.outerTable.row = e.outerTable.iter.Next() e.innerIter4Row.Begin() } From af7fed9b3848f71db23d3763038e3e29e0aa9b60 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Wed, 8 Aug 2018 15:38:37 +0800 Subject: [PATCH 17/87] executor: fix group_concat(a) when a is null (#7287) --- executor/aggfuncs/func_group_concat.go | 17 ++++++++++++++--- executor/aggregate_test.go | 5 +++++ 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index 7e7e032f4ba11..ecb22cd418daf 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -103,8 +103,11 @@ func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ p.buffer.WriteString(e.sep) } } - p.buffer.Truncate(p.buffer.Len() - len(e.sep)) - return e.truncatePartialResultIfNeed(sctx, p.buffer) + if p.buffer != nil { + p.buffer.Truncate(p.buffer.Len() - len(e.sep)) + return e.truncatePartialResultIfNeed(sctx, p.buffer) + } + return nil } type partialResult4GroupConcatDistinct struct { @@ -133,6 +136,7 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI p := (*partialResult4GroupConcatDistinct)(pr) v, isNull := "", false for _, row := range rowsInGroup { + allIsNull := true p.valsBuf.Reset() for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) @@ -142,8 +146,12 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI if isNull { continue } + allIsNull = false p.valsBuf.WriteString(v) } + if allIsNull { + continue + } joinedVals := p.valsBuf.String() if p.valSet.exist(joinedVals) { continue @@ -158,5 +166,8 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI // write values p.buffer.WriteString(joinedVals) } - return e.truncatePartialResultIfNeed(sctx, p.buffer) + if p.buffer != nil { + return e.truncatePartialResultIfNeed(sctx, p.buffer) + } + return nil } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index b85914532ab50..5b16781b93b87 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -320,6 +320,11 @@ func (s *testSuite) TestAggregation(c *C) { result.Check(testkit.Rows(string([]byte{0x0}), string([]byte{0x1}))) result = tk.MustQuery("select cast(a as signed) as idx, cast(max(a) as signed), cast(min(a) as signed) from t group by 1 order by idx") result.Check(testkit.Rows("0 0 0", "1 1 1")) + + tk.MustExec("drop table t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t value(null)") + tk.MustQuery("select group_concat(a), group_concat(distinct a) from t").Check(testkit.Rows(" ")) } func (s *testSuite) TestStreamAggPushDown(c *C) { From 359df6ecc2d19bf38876776d1a524cd229be0685 Mon Sep 17 00:00:00 2001 From: winkyao Date: Wed, 8 Aug 2018 17:16:05 +0800 Subject: [PATCH 18/87] variable, ddl: add session variable 'tidb_ddl_reorg_priority' to set operation priority of ddl reorg (#7116) --- ddl/ddl_api.go | 1 + ddl/index.go | 14 +++++++++----- ddl/reorg.go | 6 +++--- executor/executor_test.go | 28 ++++++++++++++++++++++++++++ meta/meta.go | 10 +++++++++- model/ddl.go | 3 +++ sessionctx/variable/session.go | 21 +++++++++++++++++++++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 4 ++++ 9 files changed, 79 insertions(+), 9 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 613d4a42a6f49..fe163c5f16007 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1934,6 +1934,7 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, unique bool, ind Type: model.ActionAddIndex, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{unique, indexName, idxColNames, indexOption}, + Priority: ctx.GetSessionVars().DDLReorgPriority, } err = d.doDDLJob(ctx, job) diff --git a/ddl/index.go b/ddl/index.go index d6b1b5394158d..623bc8b7f313b 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -468,6 +468,7 @@ type addIndexWorker struct { table table.Table colFieldMap map[int64]*types.FieldType closed bool + priority int // The following attributes are used to reduce memory allocation. defaultVals []types.Datum @@ -507,6 +508,7 @@ func newAddIndexWorker(sessCtx sessionctx.Context, worker *worker, id int, t tab index: index, table: t, colFieldMap: colFieldMap, + priority: kv.PriorityLow, defaultVals: make([]types.Datum, len(t.Cols())), rowMap: make(map[int64]types.Datum, len(colFieldMap)), } @@ -588,7 +590,7 @@ func (w *addIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgInde // taskDone means that the added handle is out of taskRange.endHandle. taskDone := false oprStartTime := startTime - err := iterateSnapshotRows(w.sessCtx.GetStore(), w.table, txn.StartTS(), taskRange.startHandle, + err := iterateSnapshotRows(w.sessCtx.GetStore(), w.priority, w.table, txn.StartTS(), taskRange.startHandle, func(handle int64, recordKey kv.Key, rawRow []byte) (bool, error) { oprEndTime := time.Now() w.logSlowOperations(oprEndTime.Sub(oprStartTime), "iterateSnapshotRows in fetchRowColVals", 0) @@ -710,7 +712,8 @@ func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (nextHan errInTxn = kv.RunInNewTxn(w.sessCtx.GetStore(), true, func(txn kv.Transaction) error { addedCount = 0 scanCount = 0 - txn.SetOption(kv.Priority, kv.PriorityLow) + txn.SetOption(kv.Priority, w.priority) + var ( idxRecords []*indexRecord err error @@ -1058,6 +1061,7 @@ func (w *worker) addPhysicalTableIndex(t table.Table, indexInfo *model.IndexInfo for i := 0; i < int(workerCnt); i++ { sessCtx := newContext(reorgInfo.d.store) idxWorkers[i] = newAddIndexWorker(sessCtx, w, i, t, indexInfo, colFieldMap) + idxWorkers[i].priority = job.Priority go idxWorkers[i].run(reorgInfo.d) } defer closeAddIndexWorkers(idxWorkers) @@ -1110,7 +1114,7 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo return true, nil } - start, end, err := getTableRange(reorg.d, t.GetPartition(pid), reorg.Job.SnapshotVer) + start, end, err := getTableRange(reorg.d, t.GetPartition(pid), reorg.Job.SnapshotVer, reorg.Job.Priority) if err != nil { return false, errors.Trace(err) } @@ -1155,11 +1159,11 @@ func allocateIndexID(tblInfo *model.TableInfo) int64 { // recordIterFunc is used for low-level record iteration. type recordIterFunc func(h int64, rowKey kv.Key, rawRecord []byte) (more bool, err error) -func iterateSnapshotRows(store kv.Storage, t table.Table, version uint64, seekHandle int64, fn recordIterFunc) error { +func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version uint64, seekHandle int64, fn recordIterFunc) error { ver := kv.Version{Ver: version} snap, err := store.GetSnapshot(ver) - snap.SetPriority(kv.PriorityLow) + snap.SetPriority(priority) if err != nil { return errors.Trace(err) } diff --git a/ddl/reorg.go b/ddl/reorg.go index 623ab8d2ecfc2..3f854c028a6a8 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -300,11 +300,11 @@ func (d *ddlCtx) GetTableMaxRowID(startTS uint64, tbl table.Table) (maxRowID int var gofailOnceGuard bool // getTableRange gets the start and end handle of a table (or partition). -func getTableRange(d *ddlCtx, tbl table.Table, snapshotVer uint64) (startHandle, endHandle int64, err error) { +func getTableRange(d *ddlCtx, tbl table.Table, snapshotVer uint64, priority int) (startHandle, endHandle int64, err error) { startHandle = math.MinInt64 endHandle = math.MaxInt64 // Get the start handle of this partition. - err = iterateSnapshotRows(d.store, tbl, snapshotVer, math.MinInt64, + err = iterateSnapshotRows(d.store, priority, tbl, snapshotVer, math.MinInt64, func(h int64, rowKey kv.Key, rawRecord []byte) (bool, error) { startHandle = h return false, nil @@ -351,7 +351,7 @@ func getReorgInfo(d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table) (*re pid = pi.Definitions[0].ID tp = tbl.(table.PartitionedTable).GetPartition(pid) } - start, end, err = getTableRange(d, tp, ver.Ver) + start, end, err = getTableRange(d, tp, ver.Ver, job.Priority) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/executor_test.go b/executor/executor_test.go index 9650bb94561e2..970482992d9b4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2211,6 +2211,34 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { cli.mu.Lock() cli.mu.checkFlags = checkRequestOff cli.mu.Unlock() + + tk.MustExec("alter table t1 drop index t1_index;") + tk.MustExec("SET SESSION tidb_ddl_reorg_priority = 'PRIORITY_NORMAL'") + + cli.mu.Lock() + cli.mu.checkFlags = checkDDLAddIndexPriority + cli.mu.Unlock() + + cli.priority = pb.CommandPri_Normal + tk.MustExec("alter table t1 add index t1_index (id);") + + cli.mu.Lock() + cli.mu.checkFlags = checkRequestOff + cli.mu.Unlock() + + tk.MustExec("alter table t1 drop index t1_index;") + tk.MustExec("SET SESSION tidb_ddl_reorg_priority = 'PRIORITY_HIGH'") + + cli.mu.Lock() + cli.mu.checkFlags = checkDDLAddIndexPriority + cli.mu.Unlock() + + cli.priority = pb.CommandPri_High + tk.MustExec("alter table t1 add index t1_index (id);") + + cli.mu.Lock() + cli.mu.checkFlags = checkRequestOff + cli.mu.Unlock() } func (s *testContextOptionSuite) TestAlterTableComment(c *C) { diff --git a/meta/meta.go b/meta/meta.go index 0f9f7a80e3622..e6c3a23fcd016 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -504,8 +504,16 @@ func (m *Meta) getDDLJob(key []byte, index int64) (*model.Job, error) { return nil, errors.Trace(err) } - job := &model.Job{} + job := &model.Job{ + // For compability, if the job is enqueued by old version TiDB and Priority field is omitted, + // set the default priority to kv.PriorityLow. + Priority: kv.PriorityLow, + } err = job.Decode(value) + // Check if the job.Priority is valid. + if job.Priority < kv.PriorityNormal || job.Priority > kv.PriorityHigh { + job.Priority = kv.PriorityLow + } return job, errors.Trace(err) } diff --git a/model/ddl.go b/model/ddl.go index b75e19ff3ea72..1492a77001930 100644 --- a/model/ddl.go +++ b/model/ddl.go @@ -163,6 +163,9 @@ type Job struct { // ReorgMeta is meta info of ddl reorganization. // This field is depreciated. ReorgMeta *DDLReorgMeta `json:"reorg_meta"` + + // Priority is only used to set the operation priority of adding indices. + Priority int `json:"priority"` } // FinishTableJob is called when a job is finished. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 298b3a0367c12..f15e4cae39c8c 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -15,6 +15,7 @@ package variable import ( "crypto/tls" + "strings" "sync" "sync/atomic" "time" @@ -284,6 +285,9 @@ type SessionVars struct { // EnableTablePartition enables table partition feature. EnableTablePartition bool + // DDLReorgPriority is the operation priority of adding indices. + DDLReorgPriority int + // EnableStreaming indicates whether the coprocessor request can use streaming API. // TODO: remove this after tidb-server configuration "enable-streaming' removed. EnableStreaming bool @@ -309,6 +313,7 @@ func NewSessionVars() *SessionVars { OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, RetryLimit: DefTiDBRetryLimit, DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -453,6 +458,20 @@ func (s *SessionVars) deleteSystemVar(name string) error { return nil } +func (s *SessionVars) setDDLReorgPriority(val string) { + val = strings.ToLower(val) + switch val { + case "priority_low": + s.DDLReorgPriority = kv.PriorityLow + case "priority_normal": + s.DDLReorgPriority = kv.PriorityNormal + case "priority_high": + s.DDLReorgPriority = kv.PriorityHigh + default: + s.DDLReorgPriority = kv.PriorityLow + } +} + // SetSystemVar sets the value of a system variable. func (s *SessionVars) SetSystemVar(name string, val string) error { switch name { @@ -554,6 +573,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableTablePartition = TiDBOptOn(val) case TiDBDDLReorgWorkerCount: SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) + case TiDBDDLReorgPriority: + s.setDDLReorgPriority(val) } s.systems[name] = val return nil diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bd38c9c77b6ad..2fe82c51c09ac 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -659,6 +659,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, {ScopeSession, TiDBConfig, ""}, {ScopeGlobal | ScopeSession, TiDBDDLReorgWorkerCount, strconv.Itoa(DefTiDBDDLReorgWorkerCount)}, + {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, } // SynonymsSysVariables is synonyms of system variables. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e4fe2c5ad7125..a08bd400237d7 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -173,6 +173,10 @@ const ( // tidb_ddl_reorg_worker_cnt defines the count of ddl reorg workers. TiDBDDLReorgWorkerCount = "tidb_ddl_reorg_worker_cnt" + + // tidb_ddl_reorg_priority defines the operations priority of adding indices. + // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH + TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" ) // Default TiDB system variable values. From 656926417647cc6a1b869d3d572becad44e0bb99 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Wed, 8 Aug 2018 18:33:09 +0800 Subject: [PATCH 19/87] rewrite CheckHandleExists to refactor partition (#7321) --- executor/write.go | 6 +--- table/tables/partition.go | 57 +++++++++++++++---------------------- table/tables/tables.go | 16 ++++++++--- table/tables/tables_test.go | 2 +- 4 files changed, 37 insertions(+), 44 deletions(-) diff --git a/executor/write.go b/executor/write.go index 3a046c12a35a2..f1bc187617d85 100644 --- a/executor/write.go +++ b/executor/write.go @@ -138,11 +138,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu skipHandleCheck := false if sc.DupKeyAsWarning { // if the new handle exists. `UPDATE IGNORE` will avoid removing record, and do nothing. - if t1, ok := t.(*tables.PartitionedTable); ok { - err = t1.CheckHandleExists(ctx, newHandle, newData) - } else { - err = tables.CheckHandleExists(ctx, t, newHandle) - } + err = tables.CheckHandleExists(ctx, t, newHandle, newData) if err != nil { return false, handleChanged, newHandle, 0, errors.Trace(err) } diff --git a/table/tables/partition.go b/table/tables/partition.go index 747c2f3a5c25c..fdf56b4f7e0a4 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -32,34 +32,34 @@ import ( log "github.com/sirupsen/logrus" ) -// Both Partition and PartitionedTable implement the table.Table interface. -var _ table.Table = &Partition{} -var _ table.Table = &PartitionedTable{} +// Both partition and partitionedTable implement the table.Table interface. +var _ table.Table = &partition{} +var _ table.Table = &partitionedTable{} -// PartitionedTable implements the table.PartitionedTable interface. -var _ table.PartitionedTable = &PartitionedTable{} +// partitionedTable implements the table.PartitionedTable interface. +var _ table.PartitionedTable = &partitionedTable{} -// Partition is a feature from MySQL: +// partition is a feature from MySQL: // See https://dev.mysql.com/doc/refman/8.0/en/partitioning.html // A partition table may contain many partitions, each partition has a unique partition // id. The underlying representation of a partition and a normal table (a table with no // partitions) is basically the same. -// Partition also implements the table.Table interface. -type Partition struct { +// partition also implements the table.Table interface. +type partition struct { tableCommon } // GetID implements table.Table GetID interface. -func (p *Partition) GetID() int64 { +func (p *partition) GetID() int64 { return p.partitionID } -// PartitionedTable implements the table.PartitionedTable interface. -// PartitionedTable is a table, it contains many Partitions. -type PartitionedTable struct { +// partitionedTable implements the table.PartitionedTable interface. +// partitionedTable is a table, it contains many Partitions. +type partitionedTable struct { Table partitionExpr *PartitionExpr - partitions map[int64]*Partition + partitions map[int64]*partition } func newPartitionedTable(tbl *Table, tblInfo *model.TableInfo) (table.Table, error) { @@ -68,10 +68,10 @@ func newPartitionedTable(tbl *Table, tblInfo *model.TableInfo) (table.Table, err return nil, errors.Trace(err) } - partitions := make(map[int64]*Partition) + partitions := make(map[int64]*partition) pi := tblInfo.GetPartitionInfo() for _, p := range pi.Definitions { - var t Partition + var t partition err = initTableCommonWithIndices(&t.tableCommon, tblInfo, p.ID, tbl.Columns, tbl.alloc) if err != nil { return nil, errors.Trace(err) @@ -79,7 +79,7 @@ func newPartitionedTable(tbl *Table, tblInfo *model.TableInfo) (table.Table, err partitions[p.ID] = &t } - return &PartitionedTable{ + return &partitionedTable{ Table: *tbl, partitionExpr: partitionExpr, partitions: partitions, @@ -144,7 +144,7 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) { } // PartitionExpr returns the partition expression. -func (t *PartitionedTable) PartitionExpr() *PartitionExpr { +func (t *partitionedTable) PartitionExpr() *PartitionExpr { return t.partitionExpr } @@ -154,7 +154,7 @@ func partitionRecordKey(pid int64, handle int64) kv.Key { } // locatePartition returns the partition ID of the input record. -func (t *PartitionedTable) locatePartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int64, error) { +func (t *partitionedTable) locatePartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int64, error) { var err error partitionExprs := t.partitionExpr.UpperBounds idx := sort.Search(len(partitionExprs), func(i int) bool { @@ -175,13 +175,13 @@ func (t *PartitionedTable) locatePartition(ctx sessionctx.Context, pi *model.Par return pi.Definitions[idx].ID, nil } -// GetPartition returns a Table, which is actually a Partition. -func (t *PartitionedTable) GetPartition(pid int64) table.Table { +// GetPartition returns a Table, which is actually a partition. +func (t *partitionedTable) GetPartition(pid int64) table.Table { return t.partitions[pid] } // AddRecord implements the AddRecord method for the table.Table interface. -func (t *PartitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { +func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { partitionInfo := t.meta.GetPartitionInfo() pid, err := t.locatePartition(ctx, partitionInfo, r) if err != nil { @@ -193,7 +193,7 @@ func (t *PartitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, sk } // RemoveRecord implements table.Table RemoveRecord interface. -func (t *PartitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { +func (t *partitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { partitionInfo := t.meta.GetPartitionInfo() pid, err := t.locatePartition(ctx, partitionInfo, r) if err != nil { @@ -207,7 +207,7 @@ func (t *PartitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []typ // UpdateRecord implements table.Table UpdateRecord interface. // `touched` means which columns are really modified, used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. -func (t *PartitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error { +func (t *partitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error { partitionInfo := t.meta.GetPartitionInfo() from, err := t.locatePartition(ctx, partitionInfo, currData) if err != nil { @@ -242,14 +242,3 @@ func (t *PartitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currDat tbl := t.GetPartition(to) return tbl.UpdateRecord(ctx, h, currData, newData, touched) } - -// CheckHandleExists check whether recordID key exists. if not exists, return nil, -// otherwise return kv.ErrKeyExists error. -func (t *PartitionedTable) CheckHandleExists(ctx sessionctx.Context, handle int64, data []types.Datum) error { - info := t.Meta().GetPartitionInfo() - pid, err := t.locatePartition(ctx, info, data) - if err != nil { - return errors.Trace(err) - } - return CheckHandleExists(ctx, t.GetPartition(pid), handle) -} diff --git a/table/tables/tables.go b/table/tables/tables.go index bc9a0081f7e87..4316c522b6915 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -45,11 +45,11 @@ import ( "golang.org/x/net/context" ) -// tableCommon is shared by both Table and Partition. +// tableCommon is shared by both Table and partition. type tableCommon struct { tableID int64 // partitionID is a unique int64 to identify a partition, it equals to tableID - // if this tableCommon struct is not a Partition. + // if this tableCommon struct is not a partition. partitionID int64 Columns []*table.Column publicColumns []*table.Column @@ -530,7 +530,7 @@ func (t *tableCommon) addIndices(ctx sessionctx.Context, recordID int64, r []typ defer txn.DelOption(kv.PresumeKeyNotExistsError) skipCheck := ctx.GetSessionVars().LightningMode || ctx.GetSessionVars().StmtCtx.BatchCheck if t.meta.PKIsHandle && !skipCheck && !skipHandleCheck { - if err := CheckHandleExists(ctx, t, recordID); err != nil { + if err := CheckHandleExists(ctx, t, recordID, nil); err != nil { return recordID, errors.Trace(err) } } @@ -965,7 +965,15 @@ func FindIndexByColName(t table.Table, name string) table.Index { // CheckHandleExists check whether recordID key exists. if not exists, return nil, // otherwise return kv.ErrKeyExists error. -func CheckHandleExists(ctx sessionctx.Context, t table.Table, recordID int64) error { +func CheckHandleExists(ctx sessionctx.Context, t table.Table, recordID int64, data []types.Datum) error { + if pt, ok := t.(*partitionedTable); ok { + info := t.Meta().GetPartitionInfo() + pid, err := pt.locatePartition(ctx, info, data) + if err != nil { + return errors.Trace(err) + } + t = pt.GetPartition(pid) + } txn := ctx.Txn() // Check key exists. recordKey := t.RecordKey(recordID) diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index f99598b7e6568..599c025e90376 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -396,7 +396,7 @@ PARTITION BY RANGE ( id ) ( c.Assert(err, IsNil) // Insert into maxvalue partition. } -// TestPartitionGetID tests Partition.GetID(). +// TestPartitionGetID tests partition.GetID(). func (ts *testSuite) TestPartitionGetID(c *C) { createTable1 := `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY RANGE ( id ) ( From cd458038f5b4c115e5c6ab0e11d78709265ce378 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 8 Aug 2018 21:55:51 +0800 Subject: [PATCH 20/87] infoschema: fix information schema data race (#7318) --- executor/executor_test.go | 4 ++++ infoschema/builder.go | 39 +++++++++++++++++++++------------------ 2 files changed, 25 insertions(+), 18 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 970482992d9b4..99a178bd9b5c3 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2137,7 +2137,9 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * } } else if req.Type == tikvrpc.CmdPrewrite { if c.priority == pb.CommandPri_Low { + c.mu.Lock() c.mu.lowPriorityCnt++ + c.mu.Unlock() } } } @@ -2206,7 +2208,9 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { cli.priority = pb.CommandPri_Low tk.MustExec("alter table t1 add index t1_index (id);") + cli.mu.RLock() c.Assert(cli.mu.lowPriorityCnt > 0, IsTrue) + cli.mu.RUnlock() cli.mu.Lock() cli.mu.checkFlags = checkRequestOff diff --git a/infoschema/builder.go b/infoschema/builder.go index 141574dcf31b9..8bd75f8a6e378 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -67,7 +67,7 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro newTableID = diff.TableID tblIDs = append(tblIDs, oldTableID) } - b.copySchemaTables(roDBInfo.Name.L) + dbInfo := b.copySchemaTables(roDBInfo.Name.L) b.copySortedTables(oldTableID, newTableID) // We try to reuse the old allocator, so the cached auto ID can be reused. @@ -83,14 +83,15 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro fmt.Sprintf("(Schema ID %d)", diff.OldSchemaID), ) } - b.applyDropTable(oldRoDBInfo, oldTableID) + oldDBInfo := b.copySchemaTables(oldRoDBInfo.Name.L) + b.applyDropTable(oldDBInfo, oldTableID) } else { - b.applyDropTable(roDBInfo, oldTableID) + b.applyDropTable(dbInfo, oldTableID) } } if tableIDIsValid(newTableID) { // All types except DropTable. - err := b.applyCreateTable(m, roDBInfo, newTableID, alloc) + err := b.applyCreateTable(m, dbInfo, newTableID, alloc) if err != nil { return nil, errors.Trace(err) } @@ -141,6 +142,7 @@ func (b *Builder) applyDropSchema(schemaID int64) []int64 { } ids := make([]int64, 0, len(di.Tables)) + di = di.Clone() for _, tbl := range di.Tables { b.applyDropTable(di, tbl.ID) // TODO: If the table ID doesn't exist. @@ -156,8 +158,8 @@ func (b *Builder) copySortedTablesBucket(bucketIdx int) { b.is.sortedTablesBuckets[bucketIdx] = newSortedTables } -func (b *Builder) applyCreateTable(m *meta.Meta, roDBInfo *model.DBInfo, tableID int64, alloc autoid.Allocator) error { - tblInfo, err := m.GetTable(roDBInfo.ID, tableID) +func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID int64, alloc autoid.Allocator) error { + tblInfo, err := m.GetTable(dbInfo.ID, tableID) if err != nil { return errors.Trace(err) } @@ -165,19 +167,19 @@ func (b *Builder) applyCreateTable(m *meta.Meta, roDBInfo *model.DBInfo, tableID // When we apply an old schema diff, the table may has been dropped already, so we need to fall back to // full load. return ErrTableNotExists.GenByArgs( - fmt.Sprintf("(Schema ID %d)", roDBInfo.ID), + fmt.Sprintf("(Schema ID %d)", dbInfo.ID), fmt.Sprintf("(Table ID %d)", tableID), ) } if alloc == nil { - schemaID := roDBInfo.ID + schemaID := dbInfo.ID alloc = autoid.NewAllocator(b.handle.store, tblInfo.GetDBID(schemaID)) } tbl, err := tables.TableFromMeta(alloc, tblInfo) if err != nil { return errors.Trace(err) } - tableNames := b.is.schemaMap[roDBInfo.Name.L] + tableNames := b.is.schemaMap[dbInfo.Name.L] tableNames.tables[tblInfo.Name.L] = tbl bucketIdx := tableBucketIdx(tableID) sortedTbls := b.is.sortedTablesBuckets[bucketIdx] @@ -187,31 +189,31 @@ func (b *Builder) applyCreateTable(m *meta.Meta, roDBInfo *model.DBInfo, tableID newTbl, ok := b.is.TableByID(tableID) if ok { - roDBInfo.Tables = append(roDBInfo.Tables, newTbl.Meta()) + dbInfo.Tables = append(dbInfo.Tables, newTbl.Meta()) } return nil } -func (b *Builder) applyDropTable(roDBInfo *model.DBInfo, tableID int64) { +func (b *Builder) applyDropTable(dbInfo *model.DBInfo, tableID int64) { bucketIdx := tableBucketIdx(tableID) sortedTbls := b.is.sortedTablesBuckets[bucketIdx] idx := sortedTbls.searchTable(tableID) if idx == -1 { return } - if tableNames, ok := b.is.schemaMap[roDBInfo.Name.L]; ok { + if tableNames, ok := b.is.schemaMap[dbInfo.Name.L]; ok { delete(tableNames.tables, sortedTbls[idx].Meta().Name.L) } // Remove the table in sorted table slice. b.is.sortedTablesBuckets[bucketIdx] = append(sortedTbls[0:idx], sortedTbls[idx+1:]...) // The old DBInfo still holds a reference to old table info, we need to remove it. - for i, tblInfo := range roDBInfo.Tables { + for i, tblInfo := range dbInfo.Tables { if tblInfo.ID == tableID { - if i == len(roDBInfo.Tables)-1 { - roDBInfo.Tables = roDBInfo.Tables[:i] + if i == len(dbInfo.Tables)-1 { + dbInfo.Tables = dbInfo.Tables[:i] } else { - roDBInfo.Tables = append(roDBInfo.Tables[:i], roDBInfo.Tables[i+1:]...) + dbInfo.Tables = append(dbInfo.Tables[:i], dbInfo.Tables[i+1:]...) } break } @@ -235,16 +237,17 @@ func (b *Builder) copySchemasMap(oldIS *infoSchema) { // copySchemaTables creates a new schemaTables instance when a table in the database has changed. // It also does modifications on the new one because old schemaTables must be read-only. -func (b *Builder) copySchemaTables(dbName string) { +func (b *Builder) copySchemaTables(dbName string) *model.DBInfo { oldSchemaTables := b.is.schemaMap[dbName] newSchemaTables := &schemaTables{ - dbInfo: oldSchemaTables.dbInfo, + dbInfo: oldSchemaTables.dbInfo.Clone(), tables: make(map[string]table.Table, len(oldSchemaTables.tables)), } for k, v := range oldSchemaTables.tables { newSchemaTables.tables[k] = v } b.is.schemaMap[dbName] = newSchemaTables + return newSchemaTables.dbInfo } // InitWithDBInfos initializes an empty new InfoSchema with a slice of DBInfo and schema version. From 18aae131bfadb2e71234f9f46ed5b26c90f28285 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 9 Aug 2018 10:02:13 +0800 Subject: [PATCH 21/87] test: fix data race in executor_test.go/checkRequestClient (#7326) --- executor/executor_test.go | 51 +++++++++++++++++++++------------------ 1 file changed, 28 insertions(+), 23 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 99a178bd9b5c3..a1d758385084a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -21,6 +21,7 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "testing" "time" @@ -2099,15 +2100,23 @@ const ( type checkRequestClient struct { tikv.Client - priority pb.CommandPri - mu struct { + priority pb.CommandPri + lowPriorityCnt uint32 + mu struct { sync.RWMutex - checkFlags uint32 - lowPriorityCnt uint32 - syncLog bool + checkFlags uint32 + syncLog bool } } +func (c *checkRequestClient) setCheckPriority(priority pb.CommandPri) { + atomic.StoreInt32((*int32)(&c.priority), int32(priority)) +} + +func (c *checkRequestClient) getCheckPriority() pb.CommandPri { + return (pb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) +} + func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { resp, err := c.Client.SendRequest(ctx, addr, req, timeout) c.mu.RLock() @@ -2116,7 +2125,7 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * if checkFlags == checkRequestPriority { switch req.Type { case tikvrpc.CmdCop: - if c.priority != req.Priority { + if c.getCheckPriority() != req.Priority { return nil, errors.New("fail to set priority") } } @@ -2132,14 +2141,12 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * } } else if checkFlags == checkDDLAddIndexPriority { if req.Type == tikvrpc.CmdScan { - if c.priority != req.Priority { + if c.getCheckPriority() != req.Priority { return nil, errors.New("fail to set priority") } } else if req.Type == tikvrpc.CmdPrewrite { - if c.priority == pb.CommandPri_Low { - c.mu.Lock() - c.mu.lowPriorityCnt++ - c.mu.Unlock() + if c.getCheckPriority() == pb.CommandPri_Low { + atomic.AddUint32(&c.lowPriorityCnt, 1) } } } @@ -2205,12 +2212,10 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.priority = pb.CommandPri_Low + cli.setCheckPriority(pb.CommandPri_Low) tk.MustExec("alter table t1 add index t1_index (id);") - cli.mu.RLock() - c.Assert(cli.mu.lowPriorityCnt > 0, IsTrue) - cli.mu.RUnlock() + c.Assert(atomic.LoadUint32(&cli.lowPriorityCnt) > 0, IsTrue) cli.mu.Lock() cli.mu.checkFlags = checkRequestOff @@ -2223,7 +2228,7 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.priority = pb.CommandPri_Normal + cli.setCheckPriority(pb.CommandPri_Normal) tk.MustExec("alter table t1 add index t1_index (id);") cli.mu.Lock() @@ -2237,7 +2242,7 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.priority = pb.CommandPri_High + cli.setCheckPriority(pb.CommandPri_High) tk.MustExec("alter table t1 add index t1_index (id);") cli.mu.Lock() @@ -2277,11 +2282,11 @@ func (s *testContextOptionSuite) TestCoprocessorPriority(c *C) { cli.mu.checkFlags = checkRequestPriority cli.mu.Unlock() - cli.priority = pb.CommandPri_High + cli.setCheckPriority(pb.CommandPri_High) tk.MustQuery("select id from t where id = 1") tk.MustQuery("select * from t1 where id = 1") - cli.priority = pb.CommandPri_Normal + cli.setCheckPriority(pb.CommandPri_Normal) tk.MustQuery("select count(*) from t") tk.MustExec("update t set id = 3") tk.MustExec("delete from t") @@ -2295,11 +2300,11 @@ func (s *testContextOptionSuite) TestCoprocessorPriority(c *C) { config.GetGlobalConfig().Log.ExpensiveThreshold = 0 defer func() { config.GetGlobalConfig().Log.ExpensiveThreshold = oldThreshold }() - cli.priority = pb.CommandPri_High + cli.setCheckPriority(pb.CommandPri_High) tk.MustQuery("select id from t where id = 1") tk.MustQuery("select * from t1 where id = 1") - cli.priority = pb.CommandPri_Low + cli.setCheckPriority(pb.CommandPri_Low) tk.MustQuery("select count(*) from t") tk.MustExec("delete from t") tk.MustExec("insert into t values (3)") @@ -2310,10 +2315,10 @@ func (s *testContextOptionSuite) TestCoprocessorPriority(c *C) { // tk.MustExec("update t set id = 2 where id = 1") // Test priority specified by SQL statement. - cli.priority = pb.CommandPri_High + cli.setCheckPriority(pb.CommandPri_High) tk.MustQuery("select HIGH_PRIORITY * from t") - cli.priority = pb.CommandPri_Low + cli.setCheckPriority(pb.CommandPri_Low) tk.MustQuery("select LOW_PRIORITY id from t where id = 1") cli.mu.Lock() From 22519aefc209db03186d714656959e4aa8785648 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 9 Aug 2018 15:48:01 +0800 Subject: [PATCH 22/87] *: update import path from coreos/gofail to etcd-io/gofail to fix ci (#7329) --- Gopkg.lock | 18 +- Gopkg.toml | 8 +- ddl/fail_db_test.go | 2 +- ddl/fail_test.go | 2 +- executor/executor_test.go | 2 +- owner/fail_test.go | 2 +- session/session_fail_test.go | 2 +- store/tikv/2pc_fail_test.go | 2 +- store/tikv/gcworker/gc_worker_test.go | 2 +- store/tikv/sql_fail_test.go | 2 +- store/tikv/store_fail_test.go | 2 +- tablecodec/tablecodec_test.go | 2 +- .../{coreos => etcd-io}/gofail/LICENSE | 0 vendor/github.com/etcd-io/gofail/NOTICE | 5 + .../github.com/etcd-io/gofail/code/binding.go | 52 +++++ .../etcd-io/gofail/code/failpoint.go | 95 +++++++++ .../github.com/etcd-io/gofail/code/rewrite.go | 167 +++++++++++++++ vendor/github.com/etcd-io/gofail/gofail.go | 195 ++++++++++++++++++ .../gofail/runtime/failpoint.go | 0 .../gofail/runtime/http.go | 0 .../gofail/runtime/runtime.go | 0 .../gofail/runtime/terms.go | 0 22 files changed, 539 insertions(+), 21 deletions(-) rename vendor/github.com/{coreos => etcd-io}/gofail/LICENSE (100%) create mode 100644 vendor/github.com/etcd-io/gofail/NOTICE create mode 100644 vendor/github.com/etcd-io/gofail/code/binding.go create mode 100644 vendor/github.com/etcd-io/gofail/code/failpoint.go create mode 100644 vendor/github.com/etcd-io/gofail/code/rewrite.go create mode 100644 vendor/github.com/etcd-io/gofail/gofail.go rename vendor/github.com/{coreos => etcd-io}/gofail/runtime/failpoint.go (100%) rename vendor/github.com/{coreos => etcd-io}/gofail/runtime/http.go (100%) rename vendor/github.com/{coreos => etcd-io}/gofail/runtime/runtime.go (100%) rename vendor/github.com/{coreos => etcd-io}/gofail/runtime/terms.go (100%) diff --git a/Gopkg.lock b/Gopkg.lock index 990c21a4f4c27..52d6fb862b56f 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -43,12 +43,6 @@ revision = "eddf599c689ec85f4752060edff5a72e81e9106a" version = "v3.2.18" -[[projects]] - branch = "master" - name = "github.com/coreos/gofail" - packages = ["runtime"] - revision = "38defe9971eda62d1b1c6a69c00acf001d46ecab" - [[projects]] name = "github.com/cznic/golex" packages = ["lex"] @@ -81,6 +75,16 @@ packages = ["."] revision = "9fdf92d4aac058959f814606bb729ed50f5e4240" +[[projects]] + branch = "master" + name = "github.com/etcd-io/gofail" + packages = [ + ".", + "code", + "runtime" + ] + revision = "51ce9a71510a58bad5ae66ddd278ef28762a1550" + [[projects]] name = "github.com/go-sql-driver/mysql" packages = ["."] @@ -420,6 +424,6 @@ [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "8edc10ad65dbfebaec4badbf0954ad89e272446001fe4e7b2e96094589520ce2" + inputs-digest = "472d8f62cb1733b34466708b19eaa0f7d80c01045ad4de7d790cfc34797c991a" solver-name = "gps-cdcl" solver-version = 1 diff --git a/Gopkg.toml b/Gopkg.toml index 2032e9e4d8b96..9f66d39fe08f6 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -28,10 +28,6 @@ required = ["github.com/golang/protobuf/jsonpb"] version = "1.1.0" name = "github.com/golang/protobuf" -[[constraint]] - branch = "master" - name = "github.com/coreos/gofail" - [[constraint]] branch = "master" name = "github.com/cznic/parser" @@ -92,3 +88,7 @@ required = ["github.com/golang/protobuf/jsonpb"] [[constraint]] name = "github.com/opentracing/basictracer-go" version = "1.0.0" + +[[constraint]] + branch = "master" + name = "github.com/etcd-io/gofail" diff --git a/ddl/fail_db_test.go b/ddl/fail_db_test.go index e855e3b5d5155..086f13af894eb 100644 --- a/ddl/fail_db_test.go +++ b/ddl/fail_db_test.go @@ -16,7 +16,7 @@ package ddl_test import ( "fmt" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/model" diff --git a/ddl/fail_test.go b/ddl/fail_test.go index eb5704d23a117..f89c4a9aae2ff 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -14,7 +14,7 @@ package ddl import ( - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/model" diff --git a/executor/executor_test.go b/executor/executor_test.go index a1d758385084a..be9460a8ab2aa 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -25,7 +25,7 @@ import ( "testing" "time" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" "github.com/juju/errors" . "github.com/pingcap/check" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" diff --git a/owner/fail_test.go b/owner/fail_test.go index 55f6a52df0aa7..99d7453518878 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/coreos/etcd/clientv3" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/logutil" diff --git a/session/session_fail_test.go b/session/session_fail_test.go index 61eff24744fc9..9cbf408eeb500 100644 --- a/session/session_fail_test.go +++ b/session/session_fail_test.go @@ -14,7 +14,7 @@ package session_test import ( - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/util/testkit" ) diff --git a/store/tikv/2pc_fail_test.go b/store/tikv/2pc_fail_test.go index 7fa688e6473a4..63701dd1b5f9c 100644 --- a/store/tikv/2pc_fail_test.go +++ b/store/tikv/2pc_fail_test.go @@ -14,7 +14,7 @@ package tikv import ( - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" "github.com/juju/errors" . "github.com/pingcap/check" "github.com/pingcap/tidb/terror" diff --git a/store/tikv/gcworker/gc_worker_test.go b/store/tikv/gcworker/gc_worker_test.go index f2d7da9b70600..f902cbc61de6d 100644 --- a/store/tikv/gcworker/gc_worker_test.go +++ b/store/tikv/gcworker/gc_worker_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/tidb/domain" diff --git a/store/tikv/sql_fail_test.go b/store/tikv/sql_fail_test.go index eec2437652c7f..4fb976c557198 100644 --- a/store/tikv/sql_fail_test.go +++ b/store/tikv/sql_fail_test.go @@ -18,7 +18,7 @@ import ( "sync" "time" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" diff --git a/store/tikv/store_fail_test.go b/store/tikv/store_fail_test.go index cdb341afe4867..6fc8ce09c9f9a 100644 --- a/store/tikv/store_fail_test.go +++ b/store/tikv/store_fail_test.go @@ -17,7 +17,7 @@ import ( "sync" "time" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "golang.org/x/net/context" ) diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 37a22cd751bc1..0c048e26e69c7 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - gofail "github.com/coreos/gofail/runtime" + gofail "github.com/etcd-io/gofail/runtime" . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/mysql" diff --git a/vendor/github.com/coreos/gofail/LICENSE b/vendor/github.com/etcd-io/gofail/LICENSE similarity index 100% rename from vendor/github.com/coreos/gofail/LICENSE rename to vendor/github.com/etcd-io/gofail/LICENSE diff --git a/vendor/github.com/etcd-io/gofail/NOTICE b/vendor/github.com/etcd-io/gofail/NOTICE new file mode 100644 index 0000000000000..23a0ada2fbb56 --- /dev/null +++ b/vendor/github.com/etcd-io/gofail/NOTICE @@ -0,0 +1,5 @@ +CoreOS Project +Copyright 2018 CoreOS, Inc + +This product includes software developed at CoreOS, Inc. +(http://www.coreos.com/). diff --git a/vendor/github.com/etcd-io/gofail/code/binding.go b/vendor/github.com/etcd-io/gofail/code/binding.go new file mode 100644 index 0000000000000..eb595b5f4202e --- /dev/null +++ b/vendor/github.com/etcd-io/gofail/code/binding.go @@ -0,0 +1,52 @@ +// Copyright 2016 CoreOS, 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package code + +import ( + "fmt" + "io" +) + +type Binding struct { + pkg string + fppath string + fps []*Failpoint +} + +func NewBinding(pkg string, fppath string, fps []*Failpoint) *Binding { + return &Binding{pkg, fppath, fps} +} + +// Write writes the fp.generated.go file for a package. +func (b *Binding) Write(dst io.Writer) error { + hdr := "// GENERATED BY GOFAIL. DO NOT EDIT.\n\n" + + "package " + b.pkg + + "\n\nimport \"github.com/etcd-io/gofail/runtime\"\n\n" + if _, err := fmt.Fprint(dst, hdr); err != nil { + return err + } + for _, fp := range b.fps { + _, err := fmt.Fprintf( + dst, + "var %s *runtime.Failpoint = runtime.NewFailpoint(%q, %q)\n", + fp.Runtime(), + b.fppath, + fp.Name()) + if err != nil { + return err + } + } + return nil +} diff --git a/vendor/github.com/etcd-io/gofail/code/failpoint.go b/vendor/github.com/etcd-io/gofail/code/failpoint.go new file mode 100644 index 0000000000000..0992ea0281fce --- /dev/null +++ b/vendor/github.com/etcd-io/gofail/code/failpoint.go @@ -0,0 +1,95 @@ +// Copyright 2016 CoreOS, 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package code + +import ( + "fmt" + "io" + "strings" +) + +type Failpoint struct { + name string + varType string + code []string + + // whitespace for padding + ws string +} + +// newFailpoint makes a new failpoint based on the a line containing a +// failpoint comment header. +func newFailpoint(l string) (*Failpoint, error) { + if !strings.HasPrefix(strings.TrimSpace(l), "// gofail:") { + // not a failpoint + return nil, nil + } + cmd := strings.SplitAfter(l, "// gofail:")[1] + fields := strings.Fields(cmd) + if len(fields) != 3 || fields[0] != "var" { + return nil, fmt.Errorf("failpoint: malformed comment header %q", l) + } + return &Failpoint{name: fields[1], varType: fields[2], ws: strings.Split(l, "//")[0]}, nil +} + +// flush writes the failpoint code to a buffer +func (fp *Failpoint) flush(dst io.Writer) error { + if len(fp.code) == 0 { + return fp.flushSingle(dst) + } + return fp.flushMulti(dst) +} + +func (fp *Failpoint) hdr(varname string) string { + hdr := fp.ws + "if v" + fp.name + ", __fpErr := " + fp.Runtime() + ".Acquire(); __fpErr == nil { " + hdr = hdr + "defer " + fp.Runtime() + ".Release(); " + if fp.varType == "struct{}" { + // unused + varname = "_" + } + return hdr + varname + ", __fpTypeOK := v" + fp.name + + ".(" + fp.varType + "); if !__fpTypeOK { goto __badType" + fp.name + "} " +} + +func (fp *Failpoint) footer() string { + return "; __badType" + fp.name + ": " + + fp.Runtime() + ".BadType(v" + fp.name + ", \"" + fp.varType + "\"); };" +} + +func (fp *Failpoint) flushSingle(dst io.Writer) error { + if _, err := io.WriteString(dst, fp.hdr("_")); err != nil { + return err + } + _, err := io.WriteString(dst, fp.footer()+"\n") + return err +} + +func (fp *Failpoint) flushMulti(dst io.Writer) error { + hdr := fp.hdr(fp.name) + "\n" + if _, err := io.WriteString(dst, hdr); err != nil { + return err + } + for _, code := range fp.code[:len(fp.code)-1] { + if _, err := io.WriteString(dst, code+"\n"); err != nil { + return err + } + } + code := fp.code[len(fp.code)-1] + _, err := io.WriteString(dst, code+fp.footer()+"\n") + return err +} + +func (fp *Failpoint) Name() string { return fp.name } +func (fp *Failpoint) Runtime() string { return "__fp_" + fp.name } diff --git a/vendor/github.com/etcd-io/gofail/code/rewrite.go b/vendor/github.com/etcd-io/gofail/code/rewrite.go new file mode 100644 index 0000000000000..d5f41299f191f --- /dev/null +++ b/vendor/github.com/etcd-io/gofail/code/rewrite.go @@ -0,0 +1,167 @@ +// Copyright 2016 CoreOS, 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package code + +import ( + "bufio" + "io" + "strings" + "unicode" +) + +// ToFailpoints turns all gofail comments into failpoint code. Returns a list of +// all failpoints it activated. +func ToFailpoints(wdst io.Writer, rsrc io.Reader) (fps []*Failpoint, err error) { + var curfp *Failpoint + + dst := bufio.NewWriter(wdst) + defer func() { + if err == nil && curfp != nil { + err = curfp.flush(dst) + } + if err == nil { + err = dst.Flush() + } + }() + + src := bufio.NewReader(rsrc) + for err == nil { + l, rerr := src.ReadString('\n') + if curfp != nil { + if strings.HasPrefix(strings.TrimSpace(l), "//") { + if len(l) > 0 && l[len(l)-1] == '\n' { + l = l[:len(l)-1] + } + curfp.code = append(curfp.code, strings.Replace(l, "//", "\t", 1)) + continue + } else { + curfp.flush(dst) + fps = append(fps, curfp) + curfp = nil + } + } else if label := gofailLabel(l); label != "" { + // expose gofail label + l = label + } else if curfp, err = newFailpoint(l); err != nil { + return + } else if curfp != nil { + // found a new failpoint + continue + } + if _, err = dst.WriteString(l); err != nil { + return + } + if rerr == io.EOF { + break + } + } + return +} + +// ToComments turns all failpoint code into GOFAIL comments. It returns +// a list of all failpoints it deactivated. +func ToComments(wdst io.Writer, rsrc io.Reader) (fps []*Failpoint, err error) { + src := bufio.NewReader(rsrc) + dst := bufio.NewWriter(wdst) + ws := "" + unmatchedBraces := 0 + for err == nil { + l, rerr := src.ReadString('\n') + err = rerr + lTrim := strings.TrimSpace(l) + + if unmatchedBraces > 0 { + opening, closing := numBraces(l) + unmatchedBraces += opening - closing + if unmatchedBraces == 0 { + // strip off badType footer + lTrim = strings.Split(lTrim, "; __badType")[0] + } + s := ws + "//" + wsPrefix(l, ws)[1:] + lTrim + "\n" + dst.WriteString(s) + continue + } + + isHdr := strings.Contains(l, ", __fpErr := __fp_") && strings.HasPrefix(lTrim, "if") + if isHdr { + ws = strings.Split(l, "i")[0] + n := strings.Split(strings.Split(l, "__fp_")[1], ".")[0] + t := strings.Split(strings.Split(l, ".(")[1], ")")[0] + dst.WriteString(ws + "// gofail: var " + n + " " + t + "\n") + if !strings.Contains(l, "; __badType") { + // not single liner + unmatchedBraces = 1 + } + fps = append(fps, &Failpoint{name: n, varType: t}) + continue + } + + if isLabel := strings.Contains(l, "\t/* gofail-label */"); isLabel { + l = strings.Replace(l, "/* gofail-label */", "// gofail:", 1) + } + + if _, werr := dst.WriteString(l); werr != nil { + return fps, werr + } + } + if err == io.EOF { + err = nil + } + dst.Flush() + return +} + +func gofailLabel(l string) string { + if !strings.HasPrefix(strings.TrimSpace(l), "// gofail:") { + return "" + } + label := strings.SplitAfter(l, "// gofail:")[1] + if len(label) == 0 || !strings.Contains(label, ":") { + return "" + } + return strings.Replace(l, "// gofail:", "/* gofail-label */", 1) +} + +func numBraces(l string) (opening int, closing int) { + for i := 0; i < len(l); i++ { + switch l[i] { + case '{': + opening++ + case '}': + closing++ + } + } + return +} + +// wsPrefix computes the left padding of a line given a whitespace prefix. +func wsPrefix(l, wsPfx string) string { + lws := "" + if len(wsPfx) == 0 { + lws = l + } else { + wsSplit := strings.SplitAfter(l, wsPfx) + if len(wsSplit) < 2 { + return "" + } + lws = strings.Join(wsSplit[1:], "") + } + for i, c := range lws { + if !unicode.IsSpace(c) { + return lws[:i] + } + } + return lws +} diff --git a/vendor/github.com/etcd-io/gofail/gofail.go b/vendor/github.com/etcd-io/gofail/gofail.go new file mode 100644 index 0000000000000..a1798fc8377fb --- /dev/null +++ b/vendor/github.com/etcd-io/gofail/gofail.go @@ -0,0 +1,195 @@ +// Copyright 2016 CoreOS, 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// gofail is a tool for enabling/disabling failpoints in go code. +package main + +import ( + "fmt" + "go/build" + "io" + "os" + "path" + "path/filepath" + "strings" + + "github.com/etcd-io/gofail/code" +) + +type xfrmFunc func(io.Writer, io.Reader) ([]*code.Failpoint, error) + +func xfrmFile(xfrm xfrmFunc, path string) ([]*code.Failpoint, error) { + src, serr := os.Open(path) + if serr != nil { + return nil, serr + } + defer src.Close() + + dst, derr := os.OpenFile(path+".tmp", os.O_WRONLY|os.O_CREATE|os.O_EXCL, 0600) + if derr != nil { + return nil, derr + } + defer dst.Close() + + fps, xerr := xfrm(dst, src) + if xerr != nil || len(fps) == 0 { + os.Remove(dst.Name()) + return nil, xerr + } + + rerr := os.Rename(dst.Name(), path) + if rerr != nil { + os.Remove(dst.Name()) + return nil, rerr + } + + return fps, nil +} + +func dir2files(dir, ext string) (ret []string, err error) { + if dir, err = filepath.Abs(dir); err != nil { + return nil, err + } + + f, ferr := os.Open(dir) + if ferr != nil { + return nil, ferr + } + defer f.Close() + + names, rerr := f.Readdirnames(0) + if rerr != nil { + return nil, rerr + } + for _, f := range names { + if path.Ext(f) != ext { + continue + } + ret = append(ret, path.Join(dir, f)) + } + return ret, nil +} + +func paths2files(paths []string) (files []string) { + // no paths => use cwd + if len(paths) == 0 { + wd, gerr := os.Getwd() + if gerr != nil { + fmt.Println(gerr) + os.Exit(1) + } + return paths2files([]string{wd}) + } + for _, p := range paths { + s, serr := os.Stat(p) + if serr != nil { + fmt.Println(serr) + os.Exit(1) + } + if s.IsDir() { + fs, err := dir2files(p, ".go") + if err != nil { + fmt.Println(err) + os.Exit(1) + } + files = append(files, fs...) + } else if path.Ext(s.Name()) == ".go" { + abs, err := filepath.Abs(p) + if err != nil { + fmt.Println(err) + os.Exit(1) + } + files = append(files, abs) + } + } + return files +} + +func writeBinding(file string, fps []*code.Failpoint) { + if len(fps) == 0 { + return + } + fname := strings.Split(path.Base(file), ".go")[0] + ".fail.go" + out, err := os.Create(path.Join(path.Dir(file), fname)) + if err != nil { + fmt.Println(err) + os.Exit(1) + } + // XXX: support "package main" + pkgAbsDir := path.Dir(file) + pkg := path.Base(pkgAbsDir) + pkgDir := "" + for _, srcdir := range build.Default.SrcDirs() { + if strings.HasPrefix(pkgAbsDir, srcdir) { + pkgDir = strings.Replace(pkgAbsDir, srcdir, "", 1) + break + } + } + fppath := pkg + if pkgDir == "" { + fmt.Fprintf( + os.Stderr, + "missing package for %q; using %q as failpoint path\n", + pkgAbsDir, + pkg) + } else { + fppath = pkgDir[1:] + } + code.NewBinding(pkg, fppath, fps).Write(out) + out.Close() +} + +func main() { + if len(os.Args) < 2 { + fmt.Println("not enough arguments") + os.Exit(1) + } + + var xfrm xfrmFunc + enable := false + switch os.Args[1] { + case "enable": + xfrm = code.ToFailpoints + enable = true + case "disable": + xfrm = code.ToComments + default: + fmt.Println("expected enable or disable") + os.Exit(1) + } + + files := paths2files(os.Args[2:]) + fps := [][]*code.Failpoint{} + for _, path := range files { + curfps, err := xfrmFile(xfrm, path) + if err != nil { + fmt.Println(err) + os.Exit(1) + } + fps = append(fps, curfps) + } + + if enable { + // build runtime bindings .fail.go + for i := range files { + writeBinding(files[i], fps[i]) + } + } else { + // remove all runtime bindings + for i := range files { + fname := strings.Split(path.Base(files[i]), ".go")[0] + ".fail.go" + os.Remove(path.Join(path.Dir(files[i]), fname)) + } + } +} diff --git a/vendor/github.com/coreos/gofail/runtime/failpoint.go b/vendor/github.com/etcd-io/gofail/runtime/failpoint.go similarity index 100% rename from vendor/github.com/coreos/gofail/runtime/failpoint.go rename to vendor/github.com/etcd-io/gofail/runtime/failpoint.go diff --git a/vendor/github.com/coreos/gofail/runtime/http.go b/vendor/github.com/etcd-io/gofail/runtime/http.go similarity index 100% rename from vendor/github.com/coreos/gofail/runtime/http.go rename to vendor/github.com/etcd-io/gofail/runtime/http.go diff --git a/vendor/github.com/coreos/gofail/runtime/runtime.go b/vendor/github.com/etcd-io/gofail/runtime/runtime.go similarity index 100% rename from vendor/github.com/coreos/gofail/runtime/runtime.go rename to vendor/github.com/etcd-io/gofail/runtime/runtime.go diff --git a/vendor/github.com/coreos/gofail/runtime/terms.go b/vendor/github.com/etcd-io/gofail/runtime/terms.go similarity index 100% rename from vendor/github.com/coreos/gofail/runtime/terms.go rename to vendor/github.com/etcd-io/gofail/runtime/terms.go From 702a116a5eaab50e2e83b8adb18318fdfc6a2cc3 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 9 Aug 2018 17:37:47 +0800 Subject: [PATCH 23/87] admin: fix admin check table false alarm in the case that index contains pkIsHandle column (#7317) --- executor/executor_test.go | 5 +++++ util/admin/admin.go | 6 +++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index be9460a8ab2aa..0fae51a6ccc6b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -246,6 +246,11 @@ func (s *testSuite) TestAdmin(c *C) { // For "checksum_with_index", we have two checksums, so the result will be 1^1 = 0. // For "checksum_without_index", we only have one checksum, so the result will be 1. res.Sort().Check(testkit.Rows("test checksum_with_index 0 2 2", "test checksum_without_index 1 1 1")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b));") + tk.MustExec("insert into t values(1, 1, 1)") + tk.MustExec("admin check table t") } func (s *testSuite) fillData(tk *testkit.TestKit, table string) { diff --git a/util/admin/admin.go b/util/admin/admin.go index 5a0a3f64a08ca..a1c6f4885488a 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -628,7 +628,11 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab data := make([]types.Datum, 0, len(cols)) for _, col := range cols { if col.IsPKHandleColumn(t.Meta()) { - data = append(data, types.NewIntDatum(handle)) + if mysql.HasUnsignedFlag(col.Flag) { + data = append(data, types.NewUintDatum(uint64(handle))) + } else { + data = append(data, types.NewIntDatum(handle)) + } } else { data = append(data, rowMap[col.ID]) } From fe6e710877a6673d367c674e9b454fc06daccd40 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 9 Aug 2018 18:58:29 +0800 Subject: [PATCH 24/87] ddl: make check all versions norma (#7319) --- ddl/syncer.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ddl/syncer.go b/ddl/syncer.go index 7549604d25e35..9d4da63e2b26f 100644 --- a/ddl/syncer.go +++ b/ddl/syncer.go @@ -225,6 +225,8 @@ func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, version int func (s *schemaVersionSyncer) OwnerUpdateGlobalVersion(ctx context.Context, version int64) error { startTime := time.Now() ver := strconv.FormatInt(version, 10) + // TODO: If the version is larger than the original global version, we need set the version. + // Otherwise, we'd better set the original global version. err := s.putKV(ctx, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerUpdateGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -337,7 +339,7 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, latestV succ = false break } - if int64(ver) != latestVer { + if int64(ver) < latestVer { if notMatchVerCnt%intervalCnt == 0 { log.Infof("[syncer] check all versions, ddl %s is not synced, current ver %v, latest version %v, continue checking", kv.Key, ver, latestVer) From 00839ceeec7f4bedf2d599d32fa79ddabfc97e48 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 9 Aug 2018 20:00:03 +0800 Subject: [PATCH 25/87] stats: support loading partition stats (#7305) --- executor/analyze_test.go | 24 ++++++---- statistics/boostrap.go | 34 ++++++------- statistics/dump.go | 20 ++++---- statistics/handle.go | 57 ++++++++++++++++++---- statistics/selectivity_test.go | 10 ++-- statistics/table.go | 88 +++++++++++++++++----------------- 6 files changed, 138 insertions(+), 95 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 05d4f7368935f..dfb1194572e3f 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -15,11 +15,10 @@ package executor_test import ( "fmt" - "strings" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testkit" ) @@ -47,14 +46,19 @@ PARTITION BY RANGE ( a ) ( c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() c.Assert(pi, NotNil) - ids := make([]string, 0, len(pi.Definitions)) + do, err := session.GetDomain(s.store) + c.Assert(err, IsNil) + handle := do.StatsHandle() for _, def := range pi.Definitions { - ids = append(ids, fmt.Sprintf("%d", def.ID)) + statsTbl := handle.GetPartitionStats(table.Meta(), def.ID) + c.Assert(statsTbl.Pseudo, IsFalse) + c.Assert(len(statsTbl.Columns), Equals, 2) + c.Assert(len(statsTbl.Indices), Equals, 1) + for _, col := range statsTbl.Columns { + c.Assert(col.Len(), Greater, 0) + } + for _, idx := range statsTbl.Indices { + c.Assert(idx.Len(), Greater, 0) + } } - result := tk.MustQuery(fmt.Sprintf("select count(distinct(table_id)) from mysql.stats_meta where table_id in (%s)", strings.Join(ids, ","))) - result.Check(testkit.Rows(fmt.Sprintf("%d", len(ids)))) - result = tk.MustQuery(fmt.Sprintf("select count(distinct(table_id)) from mysql.stats_histograms where table_id in (%s)", strings.Join(ids, ","))) - result.Check(testkit.Rows(fmt.Sprintf("%d", len(ids)))) - result = tk.MustQuery(fmt.Sprintf("select count(distinct(table_id)) from mysql.stats_buckets where table_id in (%s)", strings.Join(ids, ","))) - result.Check(testkit.Rows(fmt.Sprintf("%d", len(ids)))) } diff --git a/statistics/boostrap.go b/statistics/boostrap.go index 170754c9214a7..0a6862394a16e 100644 --- a/statistics/boostrap.go +++ b/statistics/boostrap.go @@ -27,30 +27,30 @@ import ( "golang.org/x/net/context" ) -func initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { - tableID := row.GetInt64(1) - table, ok := is.TableByID(tableID) + physicalID := row.GetInt64(1) + table, ok := h.getTableByPhysicalID(is, physicalID) if !ok { - log.Debugf("Unknown table ID %d in stats meta table, maybe it has been dropped", tableID) + log.Debugf("Unknown physical ID %d in stats meta table, maybe it has been dropped", physicalID) continue } tableInfo := table.Meta() newHistColl := HistColl{ - TableID: tableInfo.ID, - HaveTblID: true, - Count: row.GetInt64(3), - ModifyCount: row.GetInt64(2), - Columns: make(map[int64]*Column, len(tableInfo.Columns)), - Indices: make(map[int64]*Index, len(tableInfo.Indices)), - colName2Idx: make(map[string]int64, len(tableInfo.Columns)), - colName2ID: make(map[string]int64, len(tableInfo.Columns)), + PhysicalID: physicalID, + HavePhysicalID: true, + Count: row.GetInt64(3), + ModifyCount: row.GetInt64(2), + Columns: make(map[int64]*Column, len(tableInfo.Columns)), + Indices: make(map[int64]*Index, len(tableInfo.Indices)), + colName2Idx: make(map[string]int64, len(tableInfo.Columns)), + colName2ID: make(map[string]int64, len(tableInfo.Columns)), } tbl := &Table{ HistColl: newHistColl, Version: row.GetUint64(0), } - tables[tableID] = tbl + tables[physicalID] = tbl } } @@ -76,19 +76,19 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { if chk.NumRows() == 0 { break } - initStatsMeta4Chunk(is, tables, iter) + h.initStatsMeta4Chunk(is, tables, iter) } return tables, nil } -func initStatsHistograms4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables statsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { table, ok := tables[row.GetInt64(0)] if !ok { continue } id, ndv, nullCount, version, totColSize := row.GetInt64(2), row.GetInt64(3), row.GetInt64(5), row.GetUint64(4), row.GetInt64(7) - tbl, _ := is.TableByID(table.TableID) + tbl, _ := h.getTableByPhysicalID(is, table.PhysicalID) if row.GetInt64(1) > 0 { var idxInfo *model.IndexInfo for _, idx := range tbl.Meta().Indices { @@ -145,7 +145,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables statsCache if chk.NumRows() == 0 { break } - initStatsHistograms4Chunk(is, tables, iter) + h.initStatsHistograms4Chunk(is, tables, iter) } return nil } diff --git a/statistics/dump.go b/statistics/dump.go index 74aa854336cf9..7beff42041579 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -58,7 +58,7 @@ func dumpJSONCol(hist *Histogram, CMSketch *CMSketch) *jsonColumn { // DumpStatsToJSON dumps statistic to json. func (h *Handle) DumpStatsToJSON(dbName string, tableInfo *model.TableInfo) (*JSONTable, error) { - tbl, err := h.tableStatsFromStorage(tableInfo, true) + tbl, err := h.tableStatsFromStorage(tableInfo, tableInfo.ID, true) if err != nil { return nil, errors.Trace(err) } @@ -101,18 +101,18 @@ func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable) } for _, col := range tbl.Columns { - err = h.SaveStatsToStorage(tbl.TableID, tbl.Count, 0, &col.Histogram, col.CMSketch, 1) + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, 1) if err != nil { return errors.Trace(err) } } for _, idx := range tbl.Indices { - err = h.SaveStatsToStorage(tbl.TableID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, 1) + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, 1) if err != nil { return errors.Trace(err) } } - err = h.SaveMetaToStorage(tbl.TableID, tbl.Count, tbl.ModifyCount) + err = h.SaveMetaToStorage(tbl.PhysicalID, tbl.Count, tbl.ModifyCount) if err != nil { return errors.Trace(err) } @@ -122,12 +122,12 @@ func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable) // LoadStatsFromJSONToTable load statistic from JSONTable and return the Table of statistic. func (h *Handle) LoadStatsFromJSONToTable(tableInfo *model.TableInfo, jsonTbl *JSONTable) (*Table, error) { newHistColl := HistColl{ - TableID: tableInfo.ID, - HaveTblID: true, - Count: jsonTbl.Count, - ModifyCount: jsonTbl.ModifyCount, - Columns: make(map[int64]*Column, len(jsonTbl.Columns)), - Indices: make(map[int64]*Index, len(jsonTbl.Indices)), + PhysicalID: tableInfo.ID, + HavePhysicalID: true, + Count: jsonTbl.Count, + ModifyCount: jsonTbl.ModifyCount, + Columns: make(map[int64]*Column, len(jsonTbl.Columns)), + Indices: make(map[int64]*Index, len(jsonTbl.Indices)), } tbl := &Table{ HistColl: newHistColl, diff --git a/statistics/handle.go b/statistics/handle.go index 8f3851f8891c4..9785b2a6ee656 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/sqlexec" log "github.com/sirupsen/logrus" ) @@ -40,6 +41,10 @@ type Handle struct { lastVersion uint64 // rateMap contains the error rate delta from feedback. rateMap errorRateDeltaMap + // pid2tid is the map from partition ID to table ID. + pid2tid map[int64]int64 + // schemaVersion is the version of information schema when `pid2tid` is built. + schemaVersion int64 } restrictedExec sqlexec.RestrictedSQLExecutor @@ -128,25 +133,27 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { deletedTableIDs := make([]int64, 0, len(rows)) for _, row := range rows { version := row.GetUint64(0) - tableID := row.GetInt64(1) + physicalID := row.GetInt64(1) modifyCount := row.GetInt64(2) count := row.GetInt64(3) lastVersion = version - table, ok := is.TableByID(tableID) + h.mu.Lock() + table, ok := h.getTableByPhysicalID(is, physicalID) + h.mu.Unlock() if !ok { - log.Debugf("Unknown table ID %d in stats meta table, maybe it has been dropped", tableID) - deletedTableIDs = append(deletedTableIDs, tableID) + log.Debugf("Unknown physical ID %d in stats meta table, maybe it has been dropped", physicalID) + deletedTableIDs = append(deletedTableIDs, physicalID) continue } tableInfo := table.Meta() - tbl, err := h.tableStatsFromStorage(tableInfo, false) + tbl, err := h.tableStatsFromStorage(tableInfo, physicalID, false) // Error is not nil may mean that there are some ddl changes on this table, we will not update it. if err != nil { log.Debugf("Error occurred when read table stats for table %s. The error message is %s.", tableInfo.Name.O, errors.ErrorStack(err)) continue } if tbl == nil { - deletedTableIDs = append(deletedTableIDs, tableID) + deletedTableIDs = append(deletedTableIDs, physicalID) continue } tbl.Version = version @@ -161,11 +168,45 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { return nil } +func (h *Handle) getTableByPhysicalID(is infoschema.InfoSchema, physicalID int64) (table.Table, bool) { + if is.SchemaMetaVersion() != h.mu.schemaVersion { + h.mu.schemaVersion = is.SchemaMetaVersion() + h.mu.pid2tid = buildPartitionID2TableID(is) + } + if id, ok := h.mu.pid2tid[physicalID]; ok { + return is.TableByID(id) + } + return is.TableByID(physicalID) +} + +func buildPartitionID2TableID(is infoschema.InfoSchema) map[int64]int64 { + mapper := make(map[int64]int64) + for _, db := range is.AllSchemas() { + tbls := db.Tables + for _, tbl := range tbls { + pi := tbl.GetPartitionInfo() + if pi == nil { + continue + } + for _, def := range pi.Definitions { + mapper[def.ID] = tbl.ID + } + } + } + return mapper +} + // GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *Table { - tbl, ok := h.statsCache.Load().(statsCache)[tblInfo.ID] + return h.GetPartitionStats(tblInfo, tblInfo.ID) +} + +// GetPartitionStats retrieves the partition stats from cache. +func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64) *Table { + tbl, ok := h.statsCache.Load().(statsCache)[pid] if !ok { tbl = PseudoTable(tblInfo) + tbl.PhysicalID = pid h.UpdateTableStats([]*Table{tbl}, nil) return tbl } @@ -185,7 +226,7 @@ func (h *Handle) copyFromOldCache() statsCache { func (h *Handle) UpdateTableStats(tables []*Table, deletedIDs []int64) { newCache := h.copyFromOldCache() for _, tbl := range tables { - id := tbl.TableID + id := tbl.PhysicalID newCache[id] = tbl } for _, id := range deletedIDs { diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 8f7d9a67310bb..49744337bbcec 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -99,11 +99,11 @@ func mockStatsHistogram(id int64, values []types.Datum, repeat int64, tp *types. func mockStatsTable(tbl *model.TableInfo, rowCount int64) *statistics.Table { histColl := statistics.HistColl{ - TableID: tbl.ID, - HaveTblID: true, - Count: rowCount, - Columns: make(map[int64]*statistics.Column, len(tbl.Columns)), - Indices: make(map[int64]*statistics.Index, len(tbl.Indices)), + PhysicalID: tbl.ID, + HavePhysicalID: true, + Count: rowCount, + Columns: make(map[int64]*statistics.Column, len(tbl.Columns)), + Indices: make(map[int64]*statistics.Index, len(tbl.Indices)), } statsTbl := &statistics.Table{ HistColl: histColl, diff --git a/statistics/table.go b/statistics/table.go index 88e0d74cf1791..7ab12882370ae 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -45,34 +45,33 @@ const ( // Table represents statistics for a table. type Table struct { HistColl - Version uint64 - PKIsHandle bool + Version uint64 } // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. type HistColl struct { - TableID int64 - HaveTblID bool - Columns map[int64]*Column - Indices map[int64]*Index - colName2Idx map[string]int64 // map column name to index id - colName2ID map[string]int64 // map column name to column id - Pseudo bool - Count int64 - ModifyCount int64 // Total modify count in a table. + PhysicalID int64 // PhysicalID is the partition id for a partitioned table, otherwise, it is the table id. + HavePhysicalID bool + Columns map[int64]*Column + Indices map[int64]*Index + colName2Idx map[string]int64 // map column name to index id + colName2ID map[string]int64 // map column name to column id + Pseudo bool + Count int64 + ModifyCount int64 // Total modify count in a table. } func (t *Table) copy() *Table { newHistColl := HistColl{ - TableID: t.TableID, - HaveTblID: t.HaveTblID, - Count: t.Count, - Columns: make(map[int64]*Column), - Indices: make(map[int64]*Index), - colName2Idx: make(map[string]int64), - colName2ID: make(map[string]int64), - Pseudo: t.Pseudo, - ModifyCount: t.ModifyCount, + PhysicalID: t.PhysicalID, + HavePhysicalID: t.HavePhysicalID, + Count: t.Count, + Columns: make(map[int64]*Column), + Indices: make(map[int64]*Index), + colName2Idx: make(map[string]int64), + colName2ID: make(map[string]int64), + Pseudo: t.Pseudo, + ModifyCount: t.ModifyCount, } for id, col := range t.Columns { newHistColl.Columns[id] = col @@ -124,7 +123,7 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *Table, tableInfo *m errorRate := ErrorRate{} if isAnalyzed(row.GetInt64(8)) { h.mu.Lock() - h.mu.rateMap.clear(table.TableID, histID, true) + h.mu.rateMap.clear(table.PhysicalID, histID, true) h.mu.Unlock() } else if idx != nil { errorRate = idx.ErrorRate @@ -134,11 +133,11 @@ func (h *Handle) indexStatsFromStorage(row chunk.Row, table *Table, tableInfo *m continue } if idx == nil || idx.LastUpdateVersion < histVer { - hg, err := h.histogramFromStorage(tableInfo.ID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0) + hg, err := h.histogramFromStorage(table.PhysicalID, histID, types.NewFieldType(mysql.TypeBlob), distinct, 1, histVer, nullCount, 0) if err != nil { return errors.Trace(err) } - cms, err := h.cmSketchFromStorage(tableInfo.ID, 1, idxInfo.ID) + cms, err := h.cmSketchFromStorage(table.PhysicalID, 1, idxInfo.ID) if err != nil { return errors.Trace(err) } @@ -164,7 +163,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * errorRate := ErrorRate{} if isAnalyzed(row.GetInt64(8)) { h.mu.Lock() - h.mu.rateMap.clear(table.TableID, histID, false) + h.mu.rateMap.clear(table.PhysicalID, histID, false) h.mu.Unlock() } else if col != nil { errorRate = col.ErrorRate @@ -184,7 +183,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) && !loadAll if notNeedLoad { - count, err := h.columnCountFromStorage(table.TableID, histID) + count, err := h.columnCountFromStorage(table.PhysicalID, histID) if err != nil { return errors.Trace(err) } @@ -204,11 +203,11 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * break } if col == nil || col.LastUpdateVersion < histVer || loadAll { - hg, err := h.histogramFromStorage(tableInfo.ID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize) + hg, err := h.histogramFromStorage(table.PhysicalID, histID, &colInfo.FieldType, distinct, 0, histVer, nullCount, totColSize) if err != nil { return errors.Trace(err) } - cms, err := h.cmSketchFromStorage(tableInfo.ID, 0, colInfo.ID) + cms, err := h.cmSketchFromStorage(table.PhysicalID, 0, colInfo.ID) if err != nil { return errors.Trace(err) } @@ -240,18 +239,18 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * } // tableStatsFromStorage loads table stats info from storage. -func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, loadAll bool) (*Table, error) { +func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID int64, loadAll bool) (*Table, error) { table, ok := h.statsCache.Load().(statsCache)[tableInfo.ID] // If table stats is pseudo, we also need to copy it, since we will use the column stats when // the average error rate of it is small. if !ok { histColl := HistColl{ - TableID: tableInfo.ID, - HaveTblID: true, - Columns: make(map[int64]*Column, len(tableInfo.Columns)), - Indices: make(map[int64]*Index, len(tableInfo.Indices)), - colName2Idx: make(map[string]int64), - colName2ID: make(map[string]int64), + PhysicalID: physicalID, + HavePhysicalID: true, + Columns: make(map[int64]*Column, len(tableInfo.Columns)), + Indices: make(map[int64]*Index, len(tableInfo.Indices)), + colName2Idx: make(map[string]int64), + colName2ID: make(map[string]int64), } table = &Table{ HistColl: histColl, @@ -261,7 +260,7 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, loadAll bool) table = table.copy() } table.Pseudo = false - selSQL := fmt.Sprintf("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag from mysql.stats_histograms where table_id = %d", tableInfo.ID) + selSQL := fmt.Sprintf("select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag from mysql.stats_histograms where table_id = %d", physicalID) rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) if err != nil { return nil, errors.Trace(err) @@ -288,7 +287,7 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, loadAll bool) // String implements Stringer interface. func (t *Table) String() string { strs := make([]string, 0, len(t.Columns)+1) - strs = append(strs, fmt.Sprintf("Table:%d Count:%d", t.TableID, t.Count)) + strs = append(strs, fmt.Sprintf("Table:%d Count:%d", t.PhysicalID, t.Count)) for _, col := range t.Columns { strs = append(strs, col.String()) } @@ -353,7 +352,7 @@ func (coll *HistColl) ColumnIsInvalid(sc *stmtctx.StatementContext, colID int64) } if col.NDV > 0 && col.Len() == 0 { sc.SetHistogramsNotLoad() - histogramNeededColumns.insert(tableColumnID{tableID: coll.TableID, columnID: colID}) + histogramNeededColumns.insert(tableColumnID{tableID: coll.PhysicalID, columnID: colID}) } return col.totalRowCount() == 0 || (col.NDV > 0 && col.Len() == 0) } @@ -533,16 +532,15 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idx *Index, // PseudoTable creates a pseudo table statistics. func PseudoTable(tblInfo *model.TableInfo) *Table { pseudoHistColl := HistColl{ - Count: pseudoRowCount, - TableID: tblInfo.ID, - HaveTblID: true, - Columns: make(map[int64]*Column, len(tblInfo.Columns)), - Indices: make(map[int64]*Index, len(tblInfo.Indices)), - Pseudo: true, + Count: pseudoRowCount, + PhysicalID: tblInfo.ID, + HavePhysicalID: true, + Columns: make(map[int64]*Column, len(tblInfo.Columns)), + Indices: make(map[int64]*Index, len(tblInfo.Indices)), + Pseudo: true, } t := &Table{ - HistColl: pseudoHistColl, - PKIsHandle: tblInfo.PKIsHandle, + HistColl: pseudoHistColl, } for _, col := range tblInfo.Columns { if col.State == model.StatePublic { From b449589a7b735bb507d24200156a2d73b16cb378 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 9 Aug 2018 22:27:48 +0800 Subject: [PATCH 26/87] Makefile: fix gofail go get repo path (#7341) --- Makefile | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index ca63c10cc02c5..9f47f35540f38 100644 --- a/Makefile +++ b/Makefile @@ -129,7 +129,7 @@ explaintest: server @cd cmd/explaintest && ./run-tests.sh -s ../../bin/tidb-server gotest: parserlib - go get github.com/coreos/gofail + go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) ifeq ("$(TRAVIS_COVERAGE)", "1") @echo "Running in TRAVIS_COVERAGE mode." @@ -146,21 +146,21 @@ endif @$(GOFAIL_DISABLE) race: parserlib - go get github.com/coreos/gofail + go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ $(GOTEST) -timeout 20m -race $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) leak: parserlib - go get github.com/coreos/gofail + go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) @export log_level=debug; \ $(GOTEST) -tags leak $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) tikv_integration_test: parserlib - go get github.com/coreos/gofail + go get github.com/etcd-io/gofail @$(GOFAIL_ENABLE) $(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; } @$(GOFAIL_DISABLE) From be2c276fe4afb3be1c4ab1b78f2f7f24f2392b82 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 9 Aug 2018 22:45:24 +0800 Subject: [PATCH 27/87] executor, plan: remove `Exists` (#7207) --- cmd/explaintest/r/explain_easy.result | 18 ++-- cmd/explaintest/r/explain_easy_stats.result | 18 ++-- cmd/explaintest/r/tpch.result | 102 ++++++++++---------- executor/builder.go | 14 --- executor/executor.go | 43 --------- plan/exhaust_physical_plans.go | 9 -- plan/expression_rewriter.go | 35 ++++++- plan/initialize.go | 12 --- plan/logical_plan_builder.go | 29 ------ plan/logical_plans.go | 6 -- plan/physical_plans.go | 6 -- plan/plan.go | 2 +- plan/rule_column_pruning.go | 5 - plan/stats.go | 9 -- plan/stringer.go | 2 - 15 files changed, 100 insertions(+), 210 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 2e096f0e4cfc1..d08e79cde7ce6 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -9,15 +9,15 @@ set @@session.tidb_opt_insubquery_unfold = 1; set @@session.tidb_opt_agg_push_down = 1; explain select * from t3 where exists (select s.a from t3 s having sum(s.a) = t3.a ); id count task operator info -Projection_13 8000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d -└─HashLeftJoin_14 8000.00 root semi join, inner:StreamAgg_30, equal:[eq(cast(test.t3.a), sel_agg_1)] - ├─Projection_15 10000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d, cast(test.t3.a) - │ └─TableReader_17 10000.00 root data:TableScan_16 - │ └─TableScan_16 10000.00 cop table:t3, range:[-inf,+inf], keep order:false, stats:pseudo - └─StreamAgg_30 1.00 root funcs:sum(col_0) - └─TableReader_31 1.00 root data:StreamAgg_22 - └─StreamAgg_22 1.00 cop funcs:sum(s.a) - └─TableScan_29 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo +Projection_12 8000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d +└─HashLeftJoin_13 8000.00 root semi join, inner:StreamAgg_29, equal:[eq(cast(test.t3.a), sel_agg_1)] + ├─Projection_14 10000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d, cast(test.t3.a) + │ └─TableReader_16 10000.00 root data:TableScan_15 + │ └─TableScan_15 10000.00 cop table:t3, range:[-inf,+inf], keep order:false, stats:pseudo + └─StreamAgg_29 1.00 root funcs:sum(col_0) + └─TableReader_30 1.00 root data:StreamAgg_21 + └─StreamAgg_21 1.00 cop funcs:sum(s.a) + └─TableScan_28 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t1; id count task operator info TableReader_5 10000.00 root data:TableScan_4 diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 740fa7b9fea4b..7aec0dda5f6fe 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -12,15 +12,15 @@ set @@session.tidb_opt_insubquery_unfold = 1; set @@session.tidb_opt_agg_push_down = 1; explain select * from t3 where exists (select s.a from t3 s having sum(s.a) = t3.a ); id count task operator info -Projection_13 1600.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d -└─HashLeftJoin_14 1600.00 root semi join, inner:StreamAgg_30, equal:[eq(cast(test.t3.a), sel_agg_1)] - ├─Projection_15 2000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d, cast(test.t3.a) - │ └─TableReader_17 2000.00 root data:TableScan_16 - │ └─TableScan_16 2000.00 cop table:t3, range:[-inf,+inf], keep order:false - └─StreamAgg_30 1.00 root funcs:sum(col_0) - └─TableReader_31 1.00 root data:StreamAgg_22 - └─StreamAgg_22 1.00 cop funcs:sum(s.a) - └─TableScan_29 2000.00 cop table:s, range:[-inf,+inf], keep order:false +Projection_12 1600.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d +└─HashLeftJoin_13 1600.00 root semi join, inner:StreamAgg_29, equal:[eq(cast(test.t3.a), sel_agg_1)] + ├─Projection_14 2000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d, cast(test.t3.a) + │ └─TableReader_16 2000.00 root data:TableScan_15 + │ └─TableScan_15 2000.00 cop table:t3, range:[-inf,+inf], keep order:false + └─StreamAgg_29 1.00 root funcs:sum(col_0) + └─TableReader_30 1.00 root data:StreamAgg_21 + └─StreamAgg_21 1.00 cop funcs:sum(s.a) + └─TableScan_28 2000.00 cop table:s, range:[-inf,+inf], keep order:false explain select * from t1; id count task operator info TableReader_5 1999.00 root data:TableScan_4 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index d435898c2b225..cdc22f2d84837 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -295,17 +295,17 @@ o_orderpriority order by o_orderpriority; id count task operator info -Sort_11 1.00 root tpch.orders.o_orderpriority:asc -└─Projection_13 1.00 root tpch.orders.o_orderpriority, 8_col_0 - └─HashAgg_16 1.00 root group by:tpch.orders.o_orderpriority, funcs:count(1), firstrow(tpch.orders.o_orderpriority) - └─IndexJoin_22 2340750.00 root semi join, inner:IndexLookUp_21, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey - ├─TableReader_34 2925937.50 root data:Selection_33 - │ └─Selection_33 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) - │ └─TableScan_32 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - └─IndexLookUp_21 240004648.80 root - ├─IndexScan_18 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - └─Selection_20 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) - └─TableScan_19 1.00 cop table:lineitem, keep order:false +Sort_10 1.00 root tpch.orders.o_orderpriority:asc +└─Projection_12 1.00 root tpch.orders.o_orderpriority, 7_col_0 + └─HashAgg_15 1.00 root group by:tpch.orders.o_orderpriority, funcs:count(1), firstrow(tpch.orders.o_orderpriority) + └─IndexJoin_21 2340750.00 root semi join, inner:IndexLookUp_20, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey + ├─TableReader_33 2925937.50 root data:Selection_32 + │ └─Selection_32 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) + │ └─TableScan_31 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + └─IndexLookUp_20 240004648.80 root + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) + └─TableScan_18 1.00 cop table:lineitem, keep order:false /* Q5 Local Supplier Volume Query This query lists the revenue volume done through local suppliers. @@ -1212,34 +1212,34 @@ numwait desc, s_name limit 100; id count task operator info -Projection_27 100.00 root tpch.supplier.s_name, 19_col_0 -└─TopN_30 100.00 root 19_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 - └─HashAgg_33 320000.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) - └─Selection_34 3786715.90 root not(18_aux_0) - └─IndexJoin_40 4733394.87 root left outer semi join, inner:IndexLookUp_39, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey) - ├─IndexJoin_84 4733394.87 root semi join, inner:IndexLookUp_83, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey) - │ ├─HashLeftJoin_90 5916743.59 root inner join, inner:TableReader_119, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] - │ │ ├─HashLeftJoin_95 147918589.81 root inner join, inner:TableReader_116, equal:[eq(l1.l_suppkey, tpch.supplier.s_suppkey)] - │ │ │ ├─IndexJoin_102 147918589.81 root inner join, inner:IndexLookUp_101, outer key:tpch.orders.o_orderkey, inner key:l1.l_orderkey - │ │ │ │ ├─TableReader_111 36517371.00 root data:Selection_110 - │ │ │ │ │ └─Selection_110 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") - │ │ │ │ │ └─TableScan_109 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false - │ │ │ │ └─IndexLookUp_101 240004648.80 root - │ │ │ │ ├─IndexScan_98 1.00 cop table:l1, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false - │ │ │ │ └─Selection_100 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) - │ │ │ │ └─TableScan_99 1.00 cop table:lineitem, keep order:false - │ │ │ └─TableReader_116 500000.00 root data:TableScan_115 - │ │ │ └─TableScan_115 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false - │ │ └─TableReader_119 1.00 root data:Selection_118 - │ │ └─Selection_118 1.00 cop eq(tpch.nation.n_name, "EGYPT") - │ │ └─TableScan_117 25.00 cop table:nation, range:[-inf,+inf], keep order:false - │ └─IndexLookUp_83 1.00 root - │ ├─IndexScan_81 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - │ └─TableScan_82 1.00 cop table:lineitem, keep order:false - └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false - └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) - └─TableScan_37 1.00 cop table:lineitem, keep order:false +Projection_25 100.00 root tpch.supplier.s_name, 17_col_0 +└─TopN_28 100.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 + └─HashAgg_31 320000.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) + └─Selection_32 3786715.90 root not(16_aux_0) + └─IndexJoin_38 4733394.87 root left outer semi join, inner:IndexLookUp_37, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey) + ├─IndexJoin_82 4733394.87 root semi join, inner:IndexLookUp_81, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey) + │ ├─HashLeftJoin_88 5916743.59 root inner join, inner:TableReader_117, equal:[eq(tpch.supplier.s_nationkey, tpch.nation.n_nationkey)] + │ │ ├─HashLeftJoin_93 147918589.81 root inner join, inner:TableReader_114, equal:[eq(l1.l_suppkey, tpch.supplier.s_suppkey)] + │ │ │ ├─IndexJoin_100 147918589.81 root inner join, inner:IndexLookUp_99, outer key:tpch.orders.o_orderkey, inner key:l1.l_orderkey + │ │ │ │ ├─TableReader_109 36517371.00 root data:Selection_108 + │ │ │ │ │ └─Selection_108 36517371.00 cop eq(tpch.orders.o_orderstatus, "F") + │ │ │ │ │ └─TableScan_107 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false + │ │ │ │ └─IndexLookUp_99 240004648.80 root + │ │ │ │ ├─IndexScan_96 1.00 cop table:l1, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ │ │ │ └─Selection_98 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) + │ │ │ │ └─TableScan_97 1.00 cop table:lineitem, keep order:false + │ │ │ └─TableReader_114 500000.00 root data:TableScan_113 + │ │ │ └─TableScan_113 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false + │ │ └─TableReader_117 1.00 root data:Selection_116 + │ │ └─Selection_116 1.00 cop eq(tpch.nation.n_name, "EGYPT") + │ │ └─TableScan_115 25.00 cop table:nation, range:[-inf,+inf], keep order:false + │ └─IndexLookUp_81 1.00 root + │ ├─IndexScan_79 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ └─TableScan_80 1.00 cop table:lineitem, keep order:false + └─IndexLookUp_37 240004648.80 root + ├─IndexScan_34 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + └─Selection_36 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) + └─TableScan_35 1.00 cop table:lineitem, keep order:false /* Q22 Global Sales Opportunity Query The Global Sales Opportunity Query identifies geographies where there are customers who may be likely to make a @@ -1287,15 +1287,15 @@ cntrycode order by cntrycode; id count task operator info -Sort_33 1.00 root custsale.cntrycode:asc -└─Projection_35 1.00 root custsale.cntrycode, 29_col_0, 29_col_1 - └─HashAgg_38 1.00 root group by:custsale.cntrycode, funcs:count(1), sum(custsale.c_acctbal), firstrow(custsale.cntrycode) - └─Projection_39 0.00 root substring(tpch.customer.c_phone, 1, 2), tpch.customer.c_acctbal - └─Selection_40 0.00 root not(27_aux_0) - └─HashLeftJoin_41 0.00 root left outer semi join, inner:TableReader_47, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - ├─Selection_42 0.00 root in(substring(tpch.customer.c_phone, 1, 2), "20", "40", "22", "30", "39", "42", "21") - │ └─TableReader_45 0.00 root data:Selection_44 - │ └─Selection_44 0.00 cop gt(tpch.customer.c_acctbal, NULL) - │ └─TableScan_43 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false - └─TableReader_47 75000000.00 root data:TableScan_46 - └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false +Sort_32 1.00 root custsale.cntrycode:asc +└─Projection_34 1.00 root custsale.cntrycode, 28_col_0, 28_col_1 + └─HashAgg_37 1.00 root group by:custsale.cntrycode, funcs:count(1), sum(custsale.c_acctbal), firstrow(custsale.cntrycode) + └─Projection_38 0.00 root substring(tpch.customer.c_phone, 1, 2), tpch.customer.c_acctbal + └─Selection_39 0.00 root not(26_aux_0) + └─HashLeftJoin_40 0.00 root left outer semi join, inner:TableReader_46, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] + ├─Selection_41 0.00 root in(substring(tpch.customer.c_phone, 1, 2), "20", "40", "22", "30", "39", "42", "21") + │ └─TableReader_44 0.00 root data:Selection_43 + │ └─Selection_43 0.00 cop gt(tpch.customer.c_acctbal, NULL) + │ └─TableScan_42 7500000.00 cop table:customer, range:[-inf,+inf], keep order:false + └─TableReader_46 75000000.00 root data:TableScan_45 + └─TableScan_45 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false diff --git a/executor/builder.go b/executor/builder.go index cf5c37e58944c..eef9a5ad658d6 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -149,8 +149,6 @@ func (b *executorBuilder) build(p plan.Plan) Executor { return b.buildTableDual(v) case *plan.PhysicalApply: return b.buildApply(v) - case *plan.PhysicalExists: - return b.buildExists(v) case *plan.PhysicalMaxOneRow: return b.buildMaxOneRow(v) case *plan.Analyze: @@ -1175,18 +1173,6 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply return e } -func (b *executorBuilder) buildExists(v *plan.PhysicalExists) Executor { - childExec := b.build(v.Children()[0]) - if b.err != nil { - b.err = errors.Trace(b.err) - return nil - } - e := &ExistsExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), childExec), - } - return e -} - func (b *executorBuilder) buildMaxOneRow(v *plan.PhysicalMaxOneRow) Executor { childExec := b.build(v.Children()[0]) if b.err != nil { diff --git a/executor/executor.go b/executor/executor.go index 38454ac9ab077..45578f21e6bad 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -40,7 +40,6 @@ import ( var ( _ Executor = &CheckTableExec{} - _ Executor = &ExistsExec{} _ Executor = &HashAggExec{} _ Executor = &LimitExec{} _ Executor = &MaxOneRowExec{} @@ -866,48 +865,6 @@ func (e *TableScanExec) Open(ctx context.Context) error { return nil } -// ExistsExec represents exists executor. -type ExistsExec struct { - baseExecutor - - evaluated bool - childResult *chunk.Chunk -} - -// Open implements the Executor Open interface. -func (e *ExistsExec) Open(ctx context.Context) error { - if err := e.baseExecutor.Open(ctx); err != nil { - return errors.Trace(err) - } - e.childResult = e.children[0].newChunk() - e.evaluated = false - return nil -} - -// Next implements the Executor Next interface. -func (e *ExistsExec) Next(ctx context.Context, chk *chunk.Chunk) error { - chk.Reset() - if !e.evaluated { - e.evaluated = true - err := e.children[0].Next(ctx, e.childResult) - if err != nil { - return errors.Trace(err) - } - if e.childResult.NumRows() > 0 { - chk.AppendInt64(0, 1) - } else { - chk.AppendInt64(0, 0) - } - } - return nil -} - -// Close implements the Executor Close interface. -func (e *ExistsExec) Close() error { - e.childResult = nil - return errors.Trace(e.baseExecutor.Close()) -} - // MaxOneRowExec checks if the number of rows that a query returns is at maximum one. // It's built from subquery expression. type MaxOneRowExec struct { diff --git a/plan/exhaust_physical_plans.go b/plan/exhaust_physical_plans.go index 906bb9ef67cb5..c23c742510bf9 100644 --- a/plan/exhaust_physical_plans.go +++ b/plan/exhaust_physical_plans.go @@ -889,15 +889,6 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { return nil } -func (p *LogicalExists) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { - if !prop.isEmpty() { - return nil - } - exists := PhysicalExists{}.init(p.ctx, p.stats, &requiredProp{expectedCnt: 1}) - exists.SetSchema(p.schema) - return []PhysicalPlan{exists} -} - func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if !prop.isEmpty() { return nil diff --git a/plan/expression_rewriter.go b/plan/expression_rewriter.go index 2cc32cde00624..b2811f8925d5c 100644 --- a/plan/expression_rewriter.go +++ b/plan/expression_rewriter.go @@ -544,9 +544,9 @@ func (er *expressionRewriter) handleExistSubquery(v *ast.ExistsSubqueryExpr) (as er.err = errors.Trace(err) return v, true } - np = er.b.buildExists(np) + np = er.popExistsSubPlan(np) if len(np.extractCorrelatedCols()) > 0 { - er.p, er.err = er.b.buildSemiApply(er.p, np.Children()[0], nil, er.asScalar, false) + er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, false) if er.err != nil || !er.asScalar { return v, true } @@ -562,13 +562,38 @@ func (er *expressionRewriter) handleExistSubquery(v *ast.ExistsSubqueryExpr) (as er.err = errors.Trace(err) return v, true } - er.ctxStack = append(er.ctxStack, &expression.Constant{ - Value: rows[0][0], - RetType: types.NewFieldType(mysql.TypeTiny)}) + if len(rows) > 0 { + er.ctxStack = append(er.ctxStack, expression.One.Clone()) + } else { + er.ctxStack = append(er.ctxStack, expression.Zero.Clone()) + } } return v, true } +// popExistsSubPlan will remove the useless plan in exist's child. +// See comments inside the method for more details. +func (er *expressionRewriter) popExistsSubPlan(p LogicalPlan) LogicalPlan { +out: + for { + switch plan := p.(type) { + // This can be removed when in exists clause, + // e.g. exists(select count(*) from t order by a) is equal to exists t. + case *LogicalProjection, *LogicalSort: + p = p.Children()[0] + case *LogicalAggregation: + if len(plan.GroupByItems) == 0 { + p = LogicalTableDual{RowCount: 1}.init(er.ctx) + break out + } + p = p.Children()[0] + default: + break out + } + } + return p +} + func (er *expressionRewriter) handleInSubquery(v *ast.PatternInExpr) (ast.Node, bool) { asScalar := er.asScalar er.asScalar = true diff --git a/plan/initialize.go b/plan/initialize.go index 7af3608ac21d2..5ae709baa7e77 100644 --- a/plan/initialize.go +++ b/plan/initialize.go @@ -194,18 +194,6 @@ func (p PhysicalTableDual) init(ctx sessionctx.Context, stats *statsInfo) *Physi return &p } -func (p LogicalExists) init(ctx sessionctx.Context) *LogicalExists { - p.baseLogicalPlan = newBaseLogicalPlan(ctx, TypeExists, &p) - return &p -} - -func (p PhysicalExists) init(ctx sessionctx.Context, stats *statsInfo, props ...*requiredProp) *PhysicalExists { - p.basePhysicalPlan = newBasePhysicalPlan(ctx, TypeExists, &p) - p.childrenReqProps = props - p.stats = stats - return &p -} - func (p LogicalMaxOneRow) init(ctx sessionctx.Context) *LogicalMaxOneRow { p.baseLogicalPlan = newBaseLogicalPlan(ctx, TypeMaxOneRow, &p) return &p diff --git a/plan/logical_plan_builder.go b/plan/logical_plan_builder.go index 6d3e539436e41..2180c68e9d9ad 100644 --- a/plan/logical_plan_builder.go +++ b/plan/logical_plan_builder.go @@ -1917,35 +1917,6 @@ func (b *planBuilder) buildSemiApply(outerPlan, innerPlan LogicalPlan, condition return ap, nil } -func (b *planBuilder) buildExists(p LogicalPlan) LogicalPlan { -out: - for { - switch plan := p.(type) { - // This can be removed when in exists clause, - // e.g. exists(select count(*) from t order by a) is equal to exists t. - case *LogicalProjection, *LogicalSort: - p = p.Children()[0] - case *LogicalAggregation: - if len(plan.GroupByItems) == 0 { - p = b.buildTableDual() - break out - } - p = p.Children()[0] - default: - break out - } - } - exists := LogicalExists{}.init(b.ctx) - exists.SetChildren(p) - newCol := &expression.Column{ - RetType: types.NewFieldType(mysql.TypeTiny), - ColName: model.NewCIStr("exists_col"), - UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), - } - exists.SetSchema(expression.NewSchema(newCol)) - return exists -} - func (b *planBuilder) buildMaxOneRow(p LogicalPlan) LogicalPlan { maxOneRow := LogicalMaxOneRow{}.init(b.ctx) maxOneRow.SetChildren(p) diff --git a/plan/logical_plans.go b/plan/logical_plans.go index cbc659de658e4..fd17388851c41 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -35,7 +35,6 @@ var ( _ LogicalPlan = &LogicalProjection{} _ LogicalPlan = &LogicalSelection{} _ LogicalPlan = &LogicalApply{} - _ LogicalPlan = &LogicalExists{} _ LogicalPlan = &LogicalMaxOneRow{} _ LogicalPlan = &LogicalTableDual{} _ LogicalPlan = &DataSource{} @@ -257,11 +256,6 @@ func (la *LogicalApply) extractCorrelatedCols() []*expression.CorrelatedColumn { return corCols } -// LogicalExists checks if a query returns result. -type LogicalExists struct { - logicalSchemaProducer -} - // LogicalMaxOneRow checks if a query returns no more than one row. type LogicalMaxOneRow struct { baseLogicalPlan diff --git a/plan/physical_plans.go b/plan/physical_plans.go index ec6520b4fa489..29df0d59de3eb 100644 --- a/plan/physical_plans.go +++ b/plan/physical_plans.go @@ -28,7 +28,6 @@ var ( _ PhysicalPlan = &PhysicalSelection{} _ PhysicalPlan = &PhysicalProjection{} _ PhysicalPlan = &PhysicalTopN{} - _ PhysicalPlan = &PhysicalExists{} _ PhysicalPlan = &PhysicalMaxOneRow{} _ PhysicalPlan = &PhysicalTableDual{} _ PhysicalPlan = &PhysicalUnionAll{} @@ -362,11 +361,6 @@ type PhysicalSelection struct { Conditions []expression.Expression } -// PhysicalExists is the physical operator of Exists. -type PhysicalExists struct { - physicalSchemaProducer -} - // PhysicalMaxOneRow is the physical operator of maxOneRow. type PhysicalMaxOneRow struct { basePhysicalPlan diff --git a/plan/plan.go b/plan/plan.go index d3b7c2eee5e06..7a25ca60aeb84 100644 --- a/plan/plan.go +++ b/plan/plan.go @@ -289,7 +289,7 @@ func (p *baseLogicalPlan) buildKeyInfo() { switch p.self.(type) { case *LogicalLock, *LogicalLimit, *LogicalSort, *LogicalSelection, *LogicalApply, *LogicalProjection: p.maxOneRow = p.children[0].MaxOneRow() - case *LogicalMaxOneRow, *LogicalExists: + case *LogicalMaxOneRow: p.maxOneRow = true } } diff --git a/plan/rule_column_pruning.go b/plan/rule_column_pruning.go index b493523e7efdd..e27aef65756e5 100644 --- a/plan/rule_column_pruning.go +++ b/plan/rule_column_pruning.go @@ -164,11 +164,6 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column) { } } -// PruneColumns implements LogicalPlan interface. -func (p *LogicalExists) PruneColumns(parentUsedCols []*expression.Column) { - p.children[0].PruneColumns(nil) -} - func (p *LogicalJoin) extractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column) { for _, eqCond := range p.EqualConditions { parentUsedCols = append(parentUsedCols, expression.ExtractColumns(eqCond)...) diff --git a/plan/stats.go b/plan/stats.go index 9546ef0bdcdd8..c8e641cd8550e 100644 --- a/plan/stats.go +++ b/plan/stats.go @@ -381,15 +381,6 @@ func getSingletonStats(len int) *statsInfo { return ret } -func (p *LogicalExists) deriveStats() (*statsInfo, error) { - _, err := p.children[0].deriveStats() - if err != nil { - return nil, errors.Trace(err) - } - p.stats = getSingletonStats(1) - return p.stats, nil -} - func (p *LogicalMaxOneRow) deriveStats() (*statsInfo, error) { _, err := p.children[0].deriveStats() if err != nil { diff --git a/plan/stringer.go b/plan/stringer.go index fdaf15c5eaac1..b33d4224a5da2 100644 --- a/plan/stringer.go +++ b/plan/stringer.go @@ -104,8 +104,6 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { strs = strs[:idx] idxs = idxs[:last] str = "Apply{" + strings.Join(children, "->") + "}" - case *LogicalExists, *PhysicalExists: - str = "Exists" case *LogicalMaxOneRow, *PhysicalMaxOneRow: str = "MaxOneRow" case *LogicalLimit, *PhysicalLimit: From 3e8815f960e966fae71259c1908858bdab53aeb9 Mon Sep 17 00:00:00 2001 From: Shen Li Date: Fri, 10 Aug 2018 10:41:56 +0800 Subject: [PATCH 28/87] variable: Change the default value of charset/collation to utfbmb4/utf8mb4_bin (#7198) --- sessionctx/variable/sysvar.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 2fe82c51c09ac..cd57cdc78f380 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" + "github.com/pingcap/tidb/util/charset" ) // ScopeFlag is for system variable whether can be changed in global/session dynamically or not. @@ -163,7 +164,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "innodb_max_undo_log_size", ""}, {ScopeGlobal | ScopeSession, "range_alloc_block_size", "4096"}, {ScopeGlobal, "connect_timeout", "10"}, - {ScopeGlobal | ScopeSession, "collation_server", "latin1_swedish_ci"}, + {ScopeGlobal | ScopeSession, "collation_server", charset.CollationUTF8}, {ScopeNone, "have_rtree_keys", "YES"}, {ScopeGlobal, "innodb_old_blocks_pct", "37"}, {ScopeGlobal, "innodb_file_format", "Antelope"}, @@ -295,7 +296,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, "query_cache_wlock_invalidate", "OFF"}, {ScopeGlobal | ScopeSession, "sql_buffer_result", "OFF"}, {ScopeGlobal | ScopeSession, "character_set_filesystem", "binary"}, - {ScopeGlobal | ScopeSession, "collation_database", "latin1_swedish_ci"}, + {ScopeGlobal | ScopeSession, "collation_database", charset.CollationUTF8}, {ScopeGlobal | ScopeSession, "auto_increment_increment", "1"}, {ScopeGlobal | ScopeSession, "max_heap_table_size", "16777216"}, {ScopeGlobal | ScopeSession, "div_precision_increment", "4"}, @@ -316,7 +317,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "innodb_purge_batch_size", "300"}, {ScopeNone, "have_profiling", "NO"}, {ScopeGlobal, "slave_checkpoint_group", "512"}, - {ScopeGlobal | ScopeSession, "character_set_client", "latin1"}, + {ScopeGlobal | ScopeSession, "character_set_client", charset.CharsetUTF8}, {ScopeNone, "slave_load_tmpdir", "/var/tmp/"}, {ScopeGlobal, "innodb_buffer_pool_dump_now", "OFF"}, {ScopeGlobal, "relay_log_purge", "ON"}, @@ -391,7 +392,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, "binlog_direct_non_transactional_updates", "OFF"}, {ScopeGlobal, "innodb_change_buffering", "all"}, {ScopeGlobal | ScopeSession, "sql_big_selects", "ON"}, - {ScopeGlobal | ScopeSession, CharacterSetResults, "latin1"}, + {ScopeGlobal | ScopeSession, CharacterSetResults, charset.CharsetUTF8}, {ScopeGlobal, "innodb_max_purge_lag_delay", "0"}, {ScopeGlobal | ScopeSession, "session_track_schema", ""}, {ScopeGlobal, "innodb_io_capacity_max", "2000"}, @@ -404,7 +405,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "innodb_buffer_pool_load_abort", "OFF"}, {ScopeGlobal | ScopeSession, "tx_isolation", "REPEATABLE-READ"}, {ScopeGlobal | ScopeSession, "transaction_isolation", "REPEATABLE-READ"}, - {ScopeGlobal | ScopeSession, "collation_connection", "latin1_swedish_ci"}, + {ScopeGlobal | ScopeSession, "collation_connection", charset.CollationUTF8}, {ScopeGlobal, "rpl_semi_sync_master_timeout", ""}, {ScopeGlobal | ScopeSession, "transaction_prealloc_size", "4096"}, {ScopeNone, "slave_skip_errors", "OFF"}, @@ -499,7 +500,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "max_points_in_geometry", ""}, {ScopeGlobal, "innodb_stats_sample_pages", "8"}, {ScopeGlobal | ScopeSession, "profiling_history_size", "15"}, - {ScopeGlobal | ScopeSession, "character_set_database", "latin1"}, + {ScopeGlobal | ScopeSession, "character_set_database", charset.CharsetUTF8}, {ScopeNone, "have_symlink", "YES"}, {ScopeGlobal | ScopeSession, "storage_engine", "InnoDB"}, {ScopeGlobal | ScopeSession, "sql_log_off", "OFF"}, @@ -578,10 +579,10 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "flush", "OFF"}, {ScopeGlobal | ScopeSession, "eq_range_index_dive_limit", "10"}, {ScopeNone, "performance_schema_events_stages_history_size", "10"}, - {ScopeGlobal | ScopeSession, "character_set_connection", "latin1"}, + {ScopeGlobal | ScopeSession, "character_set_connection", charset.CharsetUTF8}, {ScopeGlobal, "myisam_use_mmap", "OFF"}, {ScopeGlobal | ScopeSession, "ndb_join_pushdown", ""}, - {ScopeGlobal | ScopeSession, "character_set_server", "latin1"}, + {ScopeGlobal | ScopeSession, "character_set_server", charset.CharsetUTF8}, {ScopeGlobal, "validate_password_special_char_count", "1"}, {ScopeNone, "performance_schema_max_thread_instances", "402"}, {ScopeGlobal, "slave_rows_search_algorithms", "TABLE_SCAN,INDEX_SCAN"}, From 07c75d49eed4b36b814fa3dbbe4c1c87fac4a08e Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Fri, 10 Aug 2018 11:33:06 +0800 Subject: [PATCH 29/87] docs: refine the pull request template (#7332) --- .github/pull_request_template.md | 80 +++++++++----------------------- 1 file changed, 22 insertions(+), 58 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index e6f27652f838e..bf578a61a1338 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -2,73 +2,37 @@ Thank you for contributing to TiDB! Please read TiDB's [CONTRIBUTING](https://github.com/pingcap/tidb/blob/master/CONTRIBUTING.md) document **BEFORE** filing this PR. --> -## What have you changed? (mandatory) +### What problem does this PR solve? - - -## What is the type of the changes? (mandatory) - - - -- New feature (non-breaking change which adds functionality) -- Improvement (non-breaking change which is an improvement to an existing feature) -- Bug fix (non-breaking change which fixes an issue) -- Breaking change (fix or feature that would cause existing functionality to not work as expected) - -## How has this PR been tested? (mandatory) - - - -## Does this PR affect documentation (docs/docs-cn) update? (mandatory) - - -## Does this PR affect tidb-ansible update? (mandatory) +### What is changed and how it works? - -## Does this PR need to be added to the release notes? (mandatory) +### Check List - -An example: -``` -release note: -// put your release notes for this PR here. + - Unit test + - Integration test + - Manual test (add detailed scripts or steps below) + - No code -action required: -// put your required action in detail here. -``` ---> +Code changes + - Has exported function/method change + - Has exported variable/fields change + - Has interface methods change + - Has persistent data change -## Refer to a related PR or issue link (optional) +Side effects -## Benchmark result if necessary (optional) + - Possible performance regression + - Increased code complexity + - Breaking backward compatibility -## Add a few positive/negative examples (optional) +Related changes + - Need to cherry-pick to the release branch + - Need to update the documentation + - Need to update the `tidb-ansible` repository + - Need to be included in the release note From 3efb8d7dfd4dbbfe1c9e9229a31298ade3553e4e Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 9 Aug 2018 21:48:12 -0600 Subject: [PATCH 30/87] util/charset: Add utf8mb4 as alias for utf8 (#7339) --- expression/integration_test.go | 4 ++-- util/charset/encoding_table.go | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 6dc31251a5354..2bb7ba30c1f31 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -876,8 +876,8 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { result.Check(testkit.Rows("'121' '0' '中文' ")) // for convert - result = tk.MustQuery(`select convert("123" using "866"), convert("123" using "binary"), convert("中文" using "binary"), convert("中文" using "utf8"), convert(cast("中文" as binary) using "utf8");`) - result.Check(testkit.Rows("123 123 中文 中文 中文")) + result = tk.MustQuery(`select convert("123" using "866"), convert("123" using "binary"), convert("中文" using "binary"), convert("中文" using "utf8"), convert("中文" using "utf8mb4"), convert(cast("中文" as binary) using "utf8");`) + result.Check(testkit.Rows("123 123 中文 中文 中文 中文")) // for insert result = tk.MustQuery(`select insert("中文", 1, 1, cast("aaa" as binary)), insert("ba", -1, 1, "aaa"), insert("ba", 1, 100, "aaa"), insert("ba", 100, 1, "aaa");`) diff --git a/util/charset/encoding_table.go b/util/charset/encoding_table.go index e488ccb346cff..37a5550b79425 100644 --- a/util/charset/encoding_table.go +++ b/util/charset/encoding_table.go @@ -42,6 +42,7 @@ var encodings = map[string]struct { "unicode-1-1-utf-8": {encoding.Nop, "utf-8"}, "utf-8": {encoding.Nop, "utf-8"}, "utf8": {encoding.Nop, "utf-8"}, + "utf8mb4": {encoding.Nop, "utf-8"}, "binary": {encoding.Nop, "binary"}, "866": {charmap.CodePage866, "ibm866"}, "cp866": {charmap.CodePage866, "ibm866"}, From 0d18ac2c18c31d7d2fcf85ab7828359df0408c7f Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Fri, 10 Aug 2018 12:30:23 +0800 Subject: [PATCH 31/87] infoschema: fix bug when apply rename table diff (#7336) --- infoschema/builder.go | 4 ++-- infoschema/infoschema_test.go | 16 ++++++++++++++++ model/model.go | 8 ++++++++ 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/infoschema/builder.go b/infoschema/builder.go index 8bd75f8a6e378..ded204fc587cf 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -76,7 +76,7 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro if oldTableID == newTableID && diff.Type != model.ActionRenameTable && diff.Type != model.ActionRebaseAutoID { alloc, _ = b.is.AllocByID(oldTableID) } - if diff.Type == model.ActionRenameTable { + if diff.Type == model.ActionRenameTable && diff.OldSchemaID != diff.SchemaID { oldRoDBInfo, ok := b.is.SchemaByID(diff.OldSchemaID) if !ok { return nil, ErrDatabaseNotExists.GenByArgs( @@ -240,7 +240,7 @@ func (b *Builder) copySchemasMap(oldIS *infoSchema) { func (b *Builder) copySchemaTables(dbName string) *model.DBInfo { oldSchemaTables := b.is.schemaMap[dbName] newSchemaTables := &schemaTables{ - dbInfo: oldSchemaTables.dbInfo.Clone(), + dbInfo: oldSchemaTables.dbInfo.Copy(), tables: make(map[string]table.Table, len(oldSchemaTables.tables)), } for k, v := range oldSchemaTables.tables { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 5952181911302..61e4ab240fbab 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -178,6 +178,22 @@ func (*testSuite) TestT(c *C) { tb, err = is.TableByName(model.NewCIStr("information_schema"), model.NewCIStr("partitions")) c.Assert(err, IsNil) c.Assert(tb, NotNil) + + err = kv.RunInNewTxn(store, true, func(txn kv.Transaction) error { + meta.NewMeta(txn).CreateTable(dbID, tblInfo) + return errors.Trace(err) + }) + c.Assert(err, IsNil) + txn, err = store.Begin() + c.Assert(err, IsNil) + _, err = builder.ApplyDiff(meta.NewMeta(txn), &model.SchemaDiff{Type: model.ActionRenameTable, SchemaID: dbID, TableID: tbID, OldSchemaID: dbID}) + c.Assert(err, IsNil) + txn.Rollback() + builder.Build() + is = handle.Get() + schema, ok = is.SchemaByID(dbID) + c.Assert(ok, IsTrue) + c.Assert(len(schema.Tables), Equals, 1) } func checkApplyCreateNonExistsSchemaDoesNotPanic(c *C, txn kv.Transaction, builder *infoschema.Builder) { diff --git a/model/model.go b/model/model.go index c95134d01be49..a7a641234a2ee 100644 --- a/model/model.go +++ b/model/model.go @@ -409,6 +409,14 @@ func (db *DBInfo) Clone() *DBInfo { return &newInfo } +// Copy shallow copies DBInfo. +func (db *DBInfo) Copy() *DBInfo { + newInfo := *db + newInfo.Tables = make([]*TableInfo, len(db.Tables)) + copy(newInfo.Tables, db.Tables) + return &newInfo +} + // CIStr is case insensitive string. type CIStr struct { O string `json:"O"` // Original string. From 593810e2590e2f4c455c6cb2a65f2a813bfe6bd9 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Fri, 10 Aug 2018 03:19:10 -0500 Subject: [PATCH 32/87] table: fix show index for the partitioned table (#7346) --- ddl/db_change_test.go | 23 +++++++++++++++++++++++ ddl/db_test.go | 12 ++++++++++++ table/tables/partition.go | 3 +++ 3 files changed, 38 insertions(+) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index ad66b651f721d..416b009d61cd0 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -515,6 +515,29 @@ func (s *testStateChangeSuite) TestShowIndex(c *C) { c.Assert(err, IsNil) callback = &ddl.TestDDLCallback{} d.(ddl.DDLForTest).SetHook(callback) + + _, err = s.se.Execute(context.Background(), "set @@tidb_enable_table_partition = 1") + c.Assert(err, IsNil) + + _, err = s.se.Execute(context.Background(), `create table tr( + id int, name varchar(50), + purchased date + ) + partition by range( year(purchased) ) ( + partition p0 values less than (1990), + partition p1 values less than (1995), + partition p2 values less than (2000), + partition p3 values less than (2005), + partition p4 values less than (2010), + partition p5 values less than (2015) + );`) + c.Assert(err, IsNil) + defer s.se.Execute(context.Background(), "drop table tr") + _, err = s.se.Execute(context.Background(), "create index idx1 on tr (purchased);") + c.Assert(err, IsNil) + result, err = s.execQuery(tk, "show index from tr;") + c.Assert(err, IsNil) + err = checkResult(result, testkit.Rows("tr 1 idx1 1 purchased A 0 BTREE ", "t 1 c2 1 c2 A 0 YES BTREE ")) } func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index 65fc9bd81a6a7..ff7d487471be9 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3141,6 +3141,18 @@ func (s *testDBSuite) TestPartitionAddIndex(c *C) { tk.MustExec("alter table partition_add_idx add index idx1 (hired)") tk.MustExec("alter table partition_add_idx add index idx2 (id, hired)") + ctx := s.tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("partition_add_idx")) + c.Assert(err, IsNil) + var idx1 table.Index + for _, idx := range t.Indices() { + if idx.Meta().Name.L == "idx1" { + idx1 = idx + break + } + } + c.Assert(idx1, NotNil) tk.MustQuery("select count(hired) from partition_add_idx use index(idx1)").Check(testkit.Rows("500")) tk.MustQuery("select count(id) from partition_add_idx use index(idx2)").Check(testkit.Rows("500")) diff --git a/table/tables/partition.go b/table/tables/partition.go index fdf56b4f7e0a4..87579dfa2d84e 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -68,6 +68,9 @@ func newPartitionedTable(tbl *Table, tblInfo *model.TableInfo) (table.Table, err return nil, errors.Trace(err) } + if err = initTableIndices(&tbl.tableCommon); err != nil { + return nil, errors.Trace(err) + } partitions := make(map[int64]*partition) pi := tblInfo.GetPartitionInfo() for _, p := range pi.Definitions { From 410246a346a02adcbfb73a04517e7e56bacddfa3 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 10 Aug 2018 18:20:25 +0800 Subject: [PATCH 33/87] *: make partitioned table use statistics (#7340) --- plan/cbo_test.go | 21 +++++++++++++++++++++ plan/logical_plan_builder.go | 18 ++++++++++++++---- plan/rule_partition_processor.go | 2 +- table/tables/partition.go | 4 ++++ 4 files changed, 40 insertions(+), 5 deletions(-) diff --git a/plan/cbo_test.go b/plan/cbo_test.go index 3afcdafb25922..875709e3c54e4 100644 --- a/plan/cbo_test.go +++ b/plan/cbo_test.go @@ -411,6 +411,14 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { testKit.MustExec("create table t3 (a int, b int)") testKit.MustExec("create index a on t3 (a)") + + testKit.MustExec("set @@session.tidb_enable_table_partition=1") + testKit.MustExec("create table t4 (a int, b int) partition by range (a) (partition p1 values less than (2), partition p2 values less than (3))") + testKit.MustExec("create index a on t4 (a)") + testKit.MustExec("create index b on t4 (b)") + testKit.MustExec("insert into t4 (a,b) values (1,1),(1,2),(1,3),(1,4),(2,5),(2,6),(2,7),(2,8)") + testKit.MustExec("analyze table t4") + tests := []struct { sql string best string @@ -451,6 +459,19 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { sql: "analyze table t2 index", best: "Analyze{Index(a),Index(b)}", }, + // Test partitioned table. + { + sql: "select * from t4 where t4.a <= 2", + best: "UnionAll{TableReader(Table(t4)->Sel([le(test.t4.a, 2)]))->TableReader(Table(t4)->Sel([le(test.t4.a, 2)]))}", + }, + { + sql: "select * from t4 where t4.b < 2", + best: "UnionAll{IndexLookUp(Index(t4.b)[[-inf,2)], Table(t4))->IndexLookUp(Index(t4.b)[[-inf,2)], Table(t4))}", + }, + { + sql: "select * from t4 where t4.a = 1 and t4.b <= 2", + best: "TableReader(Table(t4)->Sel([eq(test.t4.a, 1) le(test.t4.b, 2)]))", + }, // TODO: Refine these tests in the future. //{ // sql: "select * from t2 where t2.a = 1 and t2.b <= 2", diff --git a/plan/logical_plan_builder.go b/plan/logical_plan_builder.go index 2180c68e9d9ad..d1536ab17ad8a 100644 --- a/plan/logical_plan_builder.go +++ b/plan/logical_plan_builder.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -1719,15 +1720,20 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { // 1. tidb-server started and statistics handle has not been initialized. // 2. table row count from statistics is zero. // 3. statistics is outdated. -func (b *planBuilder) getStatsTable(tblInfo *model.TableInfo) *statistics.Table { - statsHandle := domain.GetDomain(b.ctx).StatsHandle() +func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) *statistics.Table { + statsHandle := domain.GetDomain(ctx).StatsHandle() // 1. tidb-server started and statistics handle has not been initialized. if statsHandle == nil { return statistics.PseudoTable(tblInfo) } - statsTbl := statsHandle.GetTableStats(tblInfo) + var statsTbl *statistics.Table + if pid != tblInfo.ID { + statsTbl = statsHandle.GetPartitionStats(tblInfo, pid) + } else { + statsTbl = statsHandle.GetTableStats(tblInfo) + } // 2. table row count from statistics is zero. if statsTbl.Count == 0 { @@ -1773,12 +1779,16 @@ func (b *planBuilder) buildDataSource(tn *ast.TableName) (LogicalPlan, error) { } else { columns = tbl.Cols() } + var statisticTable *statistics.Table + if _, ok := tbl.(table.PartitionedTable); !ok { + statisticTable = getStatsTable(b.ctx, tbl.Meta(), tbl.Meta().ID) + } ds := DataSource{ DBName: dbName, table: tbl, tableInfo: tableInfo, - statisticTable: b.getStatsTable(tableInfo), + statisticTable: statisticTable, indexHints: tn.IndexHints, possibleAccessPaths: possiblePaths, Columns: make([]*model.ColumnInfo, 0, len(columns)), diff --git a/plan/rule_partition_processor.go b/plan/rule_partition_processor.go index f906a8526c64f..0276d16a427f0 100644 --- a/plan/rule_partition_processor.go +++ b/plan/rule_partition_processor.go @@ -109,7 +109,7 @@ func (s *partitionProcessor) prune(ds *DataSource) (LogicalPlan, error) { // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. newDataSource.id = ds.id - + newDataSource.statisticTable = getStatsTable(ds.context(), ds.table.Meta(), pi.Definitions[i].ID) children = append(children, &newDataSource) } if len(children) == 0 { diff --git a/table/tables/partition.go b/table/tables/partition.go index 87579dfa2d84e..5219f5f42c048 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -245,3 +245,7 @@ func (t *partitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currDat tbl := t.GetPartition(to) return tbl.UpdateRecord(ctx, h, currData, newData, touched) } + +func (t *partitionedTable) GetID() int64 { + panic("GetID() should never be called on PartitionedTable") +} From e3952d6eb845a355366f57e85cc582442e85236b Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Fri, 10 Aug 2018 21:06:20 +0800 Subject: [PATCH 34/87] *: collect execution details and output them in slow query log (#7302) --- distsql/distsql_test.go | 10 ++- distsql/select_result.go | 4 +- executor/adapter.go | 8 +-- kv/kv.go | 3 + sessionctx/stmtctx/stmtctx.go | 24 +++++++ store/tikv/coprocessor.go | 100 +++++++++++++++------------ util/execdetails/execdetails.go | 54 +++++++++++++++ util/execdetails/execdetails_test.go | 38 ++++++++++ 8 files changed, 188 insertions(+), 53 deletions(-) create mode 100644 util/execdetails/execdetails.go create mode 100644 util/execdetails/execdetails_test.go diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 57e230ff875a4..c933debaa52de 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tipb/go-tipb" "golang.org/x/net/context" ) @@ -204,12 +205,17 @@ func (resp *mockResponse) Next(ctx context.Context) (kv.ResultSubset, error) { // Used only for test. type mockResultSubset struct{ data []byte } -// GetData implements kv.Response interface. +// GetData implements kv.ResultSubset interface. func (r *mockResultSubset) GetData() []byte { return r.data } -// GetStartKey implements kv.Response interface. +// GetStartKey implements kv.ResultSubset interface. func (r *mockResultSubset) GetStartKey() kv.Key { return nil } +// GetExecDetails implements kv.ResultSubset interface. +func (r *mockResultSubset) GetExecDetails() *execdetails.ExecDetails { + return &execdetails.ExecDetails{} +} + func populateBuffer() []byte { numCols := 4 numRows := 1024 diff --git a/distsql/select_result.go b/distsql/select_result.go index 835104090594a..215b114e7b743 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -159,11 +159,13 @@ func (r *selectResult) getSelectResp() error { if err := r.selectResp.Error; err != nil { return terror.ClassTiKV.New(terror.ErrCode(err.Code), err.Msg) } + sc := r.ctx.GetSessionVars().StmtCtx for _, warning := range r.selectResp.Warnings { - r.ctx.GetSessionVars().StmtCtx.AppendWarning(terror.ClassTiKV.New(terror.ErrCode(warning.Code), warning.Msg)) + sc.AppendWarning(terror.ClassTiKV.New(terror.ErrCode(warning.Code), warning.Msg)) } r.feedback.Update(re.result.GetStartKey(), r.selectResp.OutputCounts) r.partialCount++ + sc.MergeExecDetails(re.result.GetExecDetails()) if len(r.selectResp.Chunks) == 0 { continue } diff --git a/executor/adapter.go b/executor/adapter.go index d211eb1289941..7b9877a5778c2 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -357,12 +357,12 @@ func (a *ExecStmt) logSlowQuery(txnTS uint64, succ bool) { user := a.Ctx.GetSessionVars().User if costTime < threshold { logutil.SlowQueryLogger.Debugf( - "[QUERY] cost_time:%v succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", - costTime, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) + "[QUERY] cost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", + costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) } else { logutil.SlowQueryLogger.Warnf( - "[SLOW_QUERY] cost_time:%v succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", - costTime, succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) + "[SLOW_QUERY] cost_time:%v %s succ:%v con:%v user:%s txn_start_ts:%v database:%v %v%vsql:%v", + costTime, sessVars.StmtCtx.GetExecDetails(), succ, connID, user, txnTS, currentDB, tableIDs, indexIDs, sql) } } diff --git a/kv/kv.go b/kv/kv.go index c9f48a50d97fa..237fbbe5f6f75 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -15,6 +15,7 @@ package kv import ( "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/util/execdetails" "golang.org/x/net/context" ) @@ -212,6 +213,8 @@ type ResultSubset interface { GetData() []byte // GetStartKey gets the start key. GetStartKey() Key + // GetExecDetails gets the detail information. + GetExecDetails() *execdetails.ExecDetails } // Response represents the response returned from KV layer. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 99cdae156dc4d..631e859940d95 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -19,6 +19,7 @@ import ( "time" "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" ) @@ -67,6 +68,7 @@ type StatementContext struct { foundRows uint64 warnings []SQLWarn histogramsNotLoad bool + execDetails execdetails.ExecDetails } // Copied from SessionVars.TimeZone. @@ -236,3 +238,25 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.warnings = nil sc.mu.Unlock() } + +// MergeExecDetails merges a single region execution details into self, used to print +// the information in slow query log. +func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails) { + sc.mu.Lock() + sc.mu.execDetails.ProcessTime += details.ProcessTime + sc.mu.execDetails.WaitTime += details.WaitTime + sc.mu.execDetails.BackoffTime += details.BackoffTime + sc.mu.execDetails.RequestCount++ + sc.mu.execDetails.TotalKeys += details.TotalKeys + sc.mu.execDetails.ProcessedKeys += details.ProcessedKeys + sc.mu.Unlock() +} + +// GetExecDetails gets the execution details for the statement. +func (sc *StatementContext) GetExecDetails() execdetails.ExecDetails { + var details execdetails.ExecDetails + sc.mu.Lock() + details = sc.mu.execDetails + sc.mu.Unlock() + return details +} diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 224f3c60a64ce..0549d619fcd36 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/goroutine_pool" "github.com/pingcap/tipb/go-tipb" log "github.com/sirupsen/logrus" @@ -103,7 +104,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variable it.concurrency = 1 } if !it.req.KeepOrder { - it.respChan = make(chan copResponse, it.concurrency) + it.respChan = make(chan *copResponse, it.concurrency) } it.open(ctx) return it @@ -114,7 +115,7 @@ type copTask struct { region RegionVerID ranges *copRanges - respChan chan copResponse + respChan chan *copResponse storeAddr string cmdType tikvrpc.CmdType } @@ -247,7 +248,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, desc bo tasks = append(tasks, &copTask{ region: region, ranges: ranges, - respChan: make(chan copResponse, 1), + respChan: make(chan *copResponse, 1), cmdType: cmdType, }) } @@ -353,7 +354,7 @@ type copIterator struct { curr int // Otherwise, results are stored in respChan. - respChan chan copResponse + respChan chan *copResponse wg sync.WaitGroup vars *kv.Variables @@ -365,7 +366,7 @@ type copIteratorWorker struct { wg *sync.WaitGroup store *tikvStore req *kv.Request - respChan chan<- copResponse + respChan chan<- *copResponse finishCh <-chan struct{} vars *kv.Variables } @@ -376,15 +377,30 @@ type copIteratorTaskSender struct { wg *sync.WaitGroup tasks []*copTask finishCh <-chan struct{} - respChan chan<- copResponse + respChan chan<- *copResponse } type copResponse struct { - *coprocessor.Response + pbResp *coprocessor.Response + execdetails.ExecDetails startKey kv.Key err error } +// GetData implements the kv.ResultSubset GetData interface. +func (rs *copResponse) GetData() []byte { + return rs.pbResp.Data +} + +// GetStartKey implements the kv.ResultSubset GetStartKey interface. +func (rs *copResponse) GetStartKey() kv.Key { + return rs.startKey +} + +func (rs *copResponse) GetExecDetails() *execdetails.ExecDetails { + return &rs.ExecDetails +} + const minLogCopTaskTime = 300 * time.Millisecond // run is a worker function that get a copTask from channel, handle it and @@ -462,7 +478,7 @@ func (sender *copIteratorTaskSender) run() { } } -func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan copResponse) (resp copResponse, ok bool, exit bool) { +func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copResponse) (resp *copResponse, ok bool, exit bool) { select { case resp, ok = <-respCh: case <-it.finishCh: @@ -486,7 +502,7 @@ func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask) (exit bool) { return } -func (worker *copIteratorWorker) sendToRespCh(resp copResponse, respCh chan<- copResponse) (exit bool) { +func (worker *copIteratorWorker) sendToRespCh(resp *copResponse, respCh chan<- *copResponse) (exit bool) { select { case respCh <- resp: case <-worker.finishCh: @@ -495,27 +511,11 @@ func (worker *copIteratorWorker) sendToRespCh(resp copResponse, respCh chan<- co return } -// copResultSubset implements the kv.ResultSubset interface. -type copResultSubset struct { - data []byte - startKey kv.Key -} - -// GetData implements the kv.ResultSubset GetData interface. -func (rs *copResultSubset) GetData() []byte { - return rs.data -} - -// GetStartKey implements the kv.ResultSubset GetStartKey interface. -func (rs *copResultSubset) GetStartKey() kv.Key { - return rs.startKey -} - // Next returns next coprocessor result. // NOTE: Use nil to indicate finish, so if the returned ResultSubset is not nil, reader should continue to call Next(). func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { var ( - resp copResponse + resp *copResponse ok bool closed bool ) @@ -556,20 +556,16 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { if err != nil { return nil, errors.Trace(err) } - - if resp.Data == nil { - return &copResultSubset{}, nil - } - return &copResultSubset{data: resp.Data, startKey: resp.startKey}, nil + return resp, nil } // handleTask handles single copTask, sends the result to channel, retry automatically on error. -func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh chan<- copResponse) { +func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh chan<- *copResponse) { remainTasks := []*copTask{task} for len(remainTasks) > 0 { tasks, err := worker.handleTaskOnce(bo, remainTasks[0], respCh) if err != nil { - resp := copResponse{err: errors.Trace(err)} + resp := &copResponse{err: errors.Trace(err)} worker.sendToRespCh(resp, respCh) return } @@ -583,7 +579,7 @@ func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. -func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { // gofail: var handleTaskOnceError bool // if handleTaskOnceError { @@ -623,7 +619,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } // Handles the response for non-streaming copTask. - return worker.handleCopResponse(bo, resp.Cop, task, ch, nil) + return worker.handleCopResponse(bo, &copResponse{pbResp: resp.Cop}, task, ch, nil) } const ( @@ -674,7 +670,7 @@ func appendScanDetail(logStr string, columnFamily string, scanInfo *kvrpcpb.Scan return logStr } -func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { defer stream.Close() var resp *coprocessor.Response var lastRange *coprocessor.KeyRange @@ -684,7 +680,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, stream *ti return nil, nil } for { - remainedTasks, err := worker.handleCopResponse(bo, resp, task, ch, lastRange) + remainedTasks, err := worker.handleCopResponse(bo, &copResponse{pbResp: resp}, task, ch, lastRange) if err != nil || len(remainedTasks) != 0 { return remainedTasks, errors.Trace(err) } @@ -710,15 +706,15 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, stream *ti // returns more tasks when that happens, or handles the response if no error. // if we're handling streaming coprocessor response, lastRange is the range of last // successful response, otherwise it's nil. -func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, resp *coprocessor.Response, task *copTask, ch chan<- copResponse, lastRange *coprocessor.KeyRange) ([]*copTask, error) { - if regionErr := resp.GetRegionError(); regionErr != nil { +func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, resp *copResponse, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange) ([]*copTask, error) { + if regionErr := resp.pbResp.GetRegionError(); regionErr != nil { if err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())); err != nil { return nil, errors.Trace(err) } // We may meet RegionError at the first packet, but not during visiting the stream. return buildCopTasks(bo, worker.store.regionCache, task.ranges, worker.req.Desc, worker.req.Streaming) } - if lockErr := resp.GetLocked(); lockErr != nil { + if lockErr := resp.pbResp.GetLocked(); lockErr != nil { log.Debugf("coprocessor encounters lock: %v", lockErr) ok, err1 := worker.store.lockResolver.ResolveLocks(bo, []*Lock{NewLock(lockErr)}) if err1 != nil { @@ -731,19 +727,31 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, resp *coproces } return worker.buildCopTasksFromRemain(bo, lastRange, task) } - if otherErr := resp.GetOtherError(); otherErr != "" { + if otherErr := resp.pbResp.GetOtherError(); otherErr != "" { err := errors.Errorf("other error: %s", otherErr) log.Warnf("coprocessor err: %v", err) return nil, errors.Trace(err) } - var startKey kv.Key // When the request is using streaming API, the `Range` is not nil. - if resp.Range != nil { - startKey = resp.Range.Start + if resp.pbResp.Range != nil { + resp.startKey = resp.pbResp.Range.Start } else { - startKey = task.ranges.at(0).StartKey + resp.startKey = task.ranges.at(0).StartKey + } + resp.BackoffTime = time.Duration(bo.totalSleep) * time.Millisecond + if pbDetails := resp.pbResp.ExecDetails; pbDetails != nil { + if handleTime := pbDetails.HandleTime; handleTime != nil { + resp.WaitTime = time.Duration(handleTime.WaitMs) * time.Millisecond + resp.ProcessTime = time.Duration(handleTime.ProcessMs) * time.Millisecond + } + if scanDetail := pbDetails.ScanDetail; scanDetail != nil { + if scanDetail.Write != nil { + resp.TotalKeys += scanDetail.Write.Total + resp.ProcessedKeys += scanDetail.Write.Processed + } + } } - worker.sendToRespCh(copResponse{resp, startKey, nil}, ch) + worker.sendToRespCh(resp, ch) return nil, nil } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go new file mode 100644 index 0000000000000..a73c5b2cd60b5 --- /dev/null +++ b/util/execdetails/execdetails.go @@ -0,0 +1,54 @@ +// Copyright 2018 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 execdetails + +import ( + "fmt" + "strings" + "time" +) + +// ExecDetails contains execution detail information. +type ExecDetails struct { + ProcessTime time.Duration + WaitTime time.Duration + BackoffTime time.Duration + RequestCount int + TotalKeys int64 + ProcessedKeys int64 +} + +// String implements the fmt.Stringer interface. +func (d ExecDetails) String() string { + parts := make([]string, 0, 6) + if d.ProcessTime > 0 { + parts = append(parts, fmt.Sprintf("process_time:%v", d.ProcessTime)) + } + if d.WaitTime > 0 { + parts = append(parts, fmt.Sprintf("wait_time:%v", d.WaitTime)) + } + if d.BackoffTime > 0 { + parts = append(parts, fmt.Sprintf("backoff_time:%v", d.BackoffTime)) + } + if d.RequestCount > 0 { + parts = append(parts, fmt.Sprintf("request_count:%d", d.RequestCount)) + } + if d.TotalKeys > 0 { + parts = append(parts, fmt.Sprintf("total_keys:%d", d.TotalKeys)) + } + if d.ProcessedKeys > 0 { + parts = append(parts, fmt.Sprintf("processed_keys:%d", d.ProcessedKeys)) + } + return strings.Join(parts, " ") +} diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go new file mode 100644 index 0000000000000..b69f2229c7668 --- /dev/null +++ b/util/execdetails/execdetails_test.go @@ -0,0 +1,38 @@ +// Copyright 2018 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 execdetails + +import ( + "testing" + "time" +) + +func TestString(t *testing.T) { + detail := &ExecDetails{ + ProcessTime: time.Second, + WaitTime: time.Second, + BackoffTime: time.Second, + RequestCount: 1, + TotalKeys: 100, + ProcessedKeys: 10, + } + expected := "process_time:1s wait_time:1s backoff_time:1s request_count:1 total_keys:100 processed_keys:10" + if str := detail.String(); str != expected { + t.Errorf("got:\n%s\nexpected:\n%s", str, expected) + } + detail = &ExecDetails{} + if str := detail.String(); str != "" { + t.Errorf("got:\n%s\nexpected:\n", str) + } +} From 51e86a3f4b40e6a21517bcf01421b4e3f679e030 Mon Sep 17 00:00:00 2001 From: lysu Date: Sat, 11 Aug 2018 11:16:15 +0800 Subject: [PATCH 35/87] variable: modify default max chunk size to 32 (#7358) --- sessionctx/variable/tidb_vars.go | 2 +- sessionctx/variable/varsutil_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index a08bd400237d7..7909cf85c1d1d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -196,7 +196,7 @@ const ( DefBatchInsert = false DefBatchDelete = false DefCurretTS = 0 - DefMaxChunkSize = 1024 + DefMaxChunkSize = 32 DefDMLBatchSize = 20000 DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB. DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index dfa7a3ab24d76..9af93aee4d61e 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -171,7 +171,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { SetSessionSystemVar(v, TiDBBatchInsert, types.NewStringDatum("1")) c.Assert(v.BatchInsert, IsTrue) - c.Assert(v.MaxChunkSize, Equals, 1024) + c.Assert(v.MaxChunkSize, Equals, 32) SetSessionSystemVar(v, TiDBMaxChunkSize, types.NewStringDatum("2")) c.Assert(v.MaxChunkSize, Equals, 2) From 87625923e0943923b00b3242a143afa02a455385 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Sun, 12 Aug 2018 00:31:01 +0800 Subject: [PATCH 36/87] sessionctx: supports `set character_set_results = null` (#7353) --- executor/set_test.go | 3 ++- sessionctx/variable/varsutil.go | 9 ++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/executor/set_test.go b/executor/set_test.go index faea6313239a4..a2c7c6c2773e1 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -133,7 +133,8 @@ func (s *testSuite) TestSetVar(c *C) { c.Assert(charset, Equals, "utf8") c.Assert(collation, Equals, "utf8_bin") - tk.MustExec("set @@character_set_results = NULL") + tk.MustExec("set character_set_results = NULL") + tk.MustQuery("select @@character_set_results").Check(testkit.Rows("")) // Test set transaction isolation level, which is equivalent to setting variable "tx_isolation". tk.MustExec("SET SESSION TRANSACTION ISOLATION LEVEL READ COMMITTED") diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 95063ba9be8fe..014b129a942e8 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -127,12 +127,11 @@ func SetSessionSystemVar(vars *SessionVars, name string, value types.Datum) erro if sysVar == nil { return UnknownSystemVar } - if value.IsNull() { - return vars.deleteSystemVar(name) - } - var sVal string + sVal := "" var err error - sVal, err = value.ToString() + if !value.IsNull() { + sVal, err = value.ToString() + } if err != nil { return errors.Trace(err) } From d1246b3c91e607861456e90613c8981dbe854732 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 13 Aug 2018 13:53:22 +0800 Subject: [PATCH 37/87] ddl: notify stats worker when truncate table (#7356) --- ddl/ddl_worker.go | 2 +- ddl/table.go | 3 ++- statistics/ddl.go | 2 +- statistics/ddl_test.go | 11 +++++++++++ statistics/handle_test.go | 2 +- 5 files changed, 16 insertions(+), 4 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 8bb6821a5a0be..326c843734e18 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -498,7 +498,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionDropForeignKey: ver, err = onDropForeignKey(t, job) case model.ActionTruncateTable: - ver, err = onTruncateTable(t, job) + ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: ver, err = onRebaseAutoID(d.store, t, job) case model.ActionRenameTable: diff --git a/ddl/table.go b/ddl/table.go index ca3c0739ebae2..66ffdaaef6428 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -194,7 +194,7 @@ func getTableInfo(t *meta.Meta, job *model.Job, schemaID int64) (*model.TableInf // onTruncateTable delete old table meta, and creates a new table identical to old table except for table ID. // As all the old data is encoded with old table ID, it can not be accessed any more. // A background job will be created to delete old data. -func onTruncateTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { schemaID := job.SchemaID tableID := job.TableID var newTableID int64 @@ -241,6 +241,7 @@ func onTruncateTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + asyncNotifyEvent(d, &util.Event{Tp: model.ActionTruncateTable, TableInfo: tblInfo}) startKey := tablecodec.EncodeTablePrefix(tableID) job.Args = []interface{}{startKey, oldPartitionIDs} return ver, nil diff --git a/statistics/ddl.go b/statistics/ddl.go index bb40f54e28e58..9c6341fe223b9 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -30,7 +30,7 @@ import ( // HandleDDLEvent begins to process a ddl task. func (h *Handle) HandleDDLEvent(t *util.Event) error { switch t.Tp { - case model.ActionCreateTable: + case model.ActionCreateTable, model.ActionTruncateTable: return h.insertTableStats2KV(t.TableInfo) case model.ActionAddColumn: return h.insertColStats2KV(t.TableInfo.ID, t.ColumnInfo) diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index 3e060e2cf8c1e..a226eb97ae9e9 100644 --- a/statistics/ddl_test.go +++ b/statistics/ddl_test.go @@ -82,6 +82,17 @@ func (s *testStatsCacheSuite) TestDDLTable(c *C) { h.Update(is) statsTbl = h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) + + testKit.MustExec("truncate table t1") + is = do.InfoSchema() + tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + tableInfo = tbl.Meta() + err = h.HandleDDLEvent(<-h.DDLEventCh()) + c.Assert(err, IsNil) + h.Update(is) + statsTbl = h.GetTableStats(tableInfo) + c.Assert(statsTbl.Pseudo, IsFalse) } func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { diff --git a/statistics/handle_test.go b/statistics/handle_test.go index 5a3148634fb86..3929ce7acffc8 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -60,10 +60,10 @@ func cleanEnv(c *C, store kv.Storage, do *domain.Domain) { tableName := tb[0] tk.MustExec(fmt.Sprintf("drop table %v", tableName)) } - do.StatsHandle().Clear() tk.MustExec("truncate table mysql.stats_meta") tk.MustExec("truncate table mysql.stats_histograms") tk.MustExec("truncate table mysql.stats_buckets") + do.StatsHandle().Clear() } func (s *testStatsCacheSuite) TestStatsCache(c *C) { From 9e1ae02df7ffbaa2ca43768e7bc63a11cacaa874 Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 13 Aug 2018 14:07:51 +0800 Subject: [PATCH 38/87] ddl: fix panic in checkRangePartitioningKeysConstraints when patition by range columns (#7366) --- ddl/db_test.go | 3 +++ ddl/partition.go | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index ff7d487471be9..69921cb423071 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1723,6 +1723,9 @@ func (s *testDBSuite) TestCreateTableWithPartition(c *C) { // test check order. The sql below have 2 problem: 1. ErrFieldTypeNotAllowedAsPartitionField 2. ErrPartitionMaxvalue , mysql will return ErrPartitionMaxvalue. s.testErrorCode(c, `create TABLE t25 (c1 float) partition by range( c1 ) (partition p1 values less than maxvalue,partition p0 values less than (2000));`, tmysql.ErrPartitionMaxvalue) + + // Fix issue 7362. + s.tk.MustExec("create table test_partition(id bigint, name varchar(255), primary key(id)) ENGINE=InnoDB DEFAULT CHARSET=utf8 PARTITION BY RANGE COLUMNS(id) (PARTITION p1 VALUES LESS THAN (10) ENGINE = InnoDB);") } func (s *testDBSuite) TestTableDDLWithFloatType(c *C) { diff --git a/ddl/partition.go b/ddl/partition.go index f9eaac5192fca..25f18d95557a9 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -285,7 +285,8 @@ func getPartitionIDs(table *model.TableInfo) []int64 { // checkRangePartitioningKeysConstraints checks that the range partitioning key is included in the table constraint. func checkRangePartitioningKeysConstraints(ctx sessionctx.Context, s *ast.CreateTableStmt, tblInfo *model.TableInfo, constraints []*ast.Constraint) error { // Returns directly if there is no constraint in the partition table. - if len(constraints) == 0 { + // TODO: Remove the test 's.Partition.Expr == nil' when we support 'PARTITION BY RANGE COLUMNS' + if len(constraints) == 0 || s.Partition.Expr == nil { return nil } From f98ef22ded66655d3d174e3fbfeb76539e6fccce Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Mon, 13 Aug 2018 14:54:06 +0800 Subject: [PATCH 39/87] expression, executor: add a new interface MergePartialResult for the new aggregation framework (#7281) --- executor/aggfuncs/aggfunc_test.go | 56 ++++ executor/aggfuncs/aggfuncs.go | 21 +- executor/aggfuncs/builder.go | 2 +- executor/aggfuncs/func_avg.go | 29 ++- executor/aggfuncs/func_avg_test.go | 347 +++++++++++++++++++++++++ executor/aggfuncs/func_bitfuncs.go | 18 ++ executor/aggfuncs/func_count.go | 6 + executor/aggfuncs/func_first_row.go | 60 +++++ executor/aggfuncs/func_group_concat.go | 15 ++ executor/aggfuncs/func_max_min.go | 143 +++++++++- executor/aggfuncs/func_sum.go | 33 ++- executor/aggregate_test.go | 5 + expression/aggregation/descriptor.go | 48 ++++ types/mydecimal.go | 2 + 14 files changed, 774 insertions(+), 11 deletions(-) create mode 100644 executor/aggfuncs/aggfunc_test.go create mode 100644 executor/aggfuncs/func_avg_test.go diff --git a/executor/aggfuncs/aggfunc_test.go b/executor/aggfuncs/aggfunc_test.go new file mode 100644 index 0000000000000..ac4c1df5f6b22 --- /dev/null +++ b/executor/aggfuncs/aggfunc_test.go @@ -0,0 +1,56 @@ +// Copyright 2018 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_test + +import ( + "testing" + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testleak" +) + +var _ = Suite(&testSuite{}) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +type testSuite struct { + *parser.Parser + ctx sessionctx.Context +} + +func (s *testSuite) SetUpSuite(c *C) { + s.Parser = parser.New() + s.ctx = mock.NewContext() + s.ctx.GetSessionVars().StmtCtx.TimeZone = time.Local +} + +func (s *testSuite) TearDownSuite(c *C) { +} + +func (s *testSuite) SetUpTest(c *C) { + s.ctx.GetSessionVars().PlanColumnID = 0 + testleak.BeforeTest() +} + +func (s *testSuite) TearDownTest(c *C) { + s.ctx.GetSessionVars().StmtCtx.SetWarnings(nil) + testleak.AfterTest(c)() +} diff --git a/executor/aggfuncs/aggfuncs.go b/executor/aggfuncs/aggfuncs.go index c493aa2112ee9..9bc105d2c1d24 100644 --- a/executor/aggfuncs/aggfuncs.go +++ b/executor/aggfuncs/aggfuncs.go @@ -34,11 +34,19 @@ var ( _ AggFunc = (*countOriginal4String)(nil) _ AggFunc = (*countOriginalWithDistinct)(nil) - // All the AggFunc implementations for "SUM" are listed here. // All the AggFunc implementations for "FIRSTROW" are listed here. + _ AggFunc = (*firstRow4Decimal)(nil) + _ AggFunc = (*firstRow4Int)(nil) + _ AggFunc = (*firstRow4Time)(nil) + _ AggFunc = (*firstRow4String)(nil) + _ AggFunc = (*firstRow4Duration)(nil) + _ AggFunc = (*firstRow4Float32)(nil) + _ AggFunc = (*firstRow4Float64)(nil) + _ AggFunc = (*firstRow4JSON)(nil) // All the AggFunc implementations for "MAX"/"MIN" are listed here. _ AggFunc = (*maxMin4Int)(nil) + _ AggFunc = (*maxMin4Uint)(nil) _ AggFunc = (*maxMin4Float32)(nil) _ AggFunc = (*maxMin4Float64)(nil) _ AggFunc = (*maxMin4Decimal)(nil) @@ -55,6 +63,7 @@ var ( _ AggFunc = (*avgPartial4Float64)(nil) _ AggFunc = (*avgOriginal4DistinctFloat64)(nil) + // All the AggFunc implementations for "SUM" are listed here. _ AggFunc = (*sum4DistinctFloat64)(nil) _ AggFunc = (*sum4DistinctDecimal)(nil) _ AggFunc = (*sum4Decimal)(nil) @@ -102,6 +111,12 @@ type AggFunc interface { // aggregate function. UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error + // MergePartialResult will be called in the final phase when parallelly + // executing. It converts the PartialResult `src`, `dst` to the same specific + // data structure which stores the partial results, and then evaluate the + // final result using the partial results as input values. + MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error + // AppendFinalResult2Chunk finalizes the partial result and append the // final result to the input chunk. Like other operations, it converts the // input PartialResult to the specific data structure which stores the @@ -119,3 +134,7 @@ type baseAggFunc struct { // used to append the final result of this function. ordinal int } + +func (*baseAggFunc) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + return nil +} diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index 3035cfb059ecb..30391b78ddb69 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -158,7 +158,7 @@ func buildAvg(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { // Build avg functions which consume the partial result of other avg // functions and update their partial results. case aggregation.Partial2Mode, aggregation.FinalMode: - switch aggFuncDesc.Args[1].GetType().Tp { + switch aggFuncDesc.RetTp.Tp { case mysql.TypeNewDecimal: return &avgPartial4Decimal{baseAvgDecimal{base}} case mysql.TypeDouble: diff --git a/executor/aggfuncs/func_avg.go b/executor/aggfuncs/func_avg.go index 2d4ac122bdcaf..033aaf1f5b84e 100644 --- a/executor/aggfuncs/func_avg.go +++ b/executor/aggfuncs/func_avg.go @@ -67,7 +67,6 @@ type avgOriginal4Decimal struct { func (e *avgOriginal4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { p := (*partialResult4AvgDecimal)(pr) - newSum := new(types.MyDecimal) for _, row := range rowsInGroup { input, isNull, err := e.args[0].EvalDecimal(sctx, row) if err != nil { @@ -77,6 +76,7 @@ func (e *avgOriginal4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsI continue } + newSum := new(types.MyDecimal) err = types.DecimalAdd(&p.sum, input, newSum) if err != nil { return errors.Trace(err) @@ -93,7 +93,6 @@ type avgPartial4Decimal struct { func (e *avgPartial4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { p := (*partialResult4AvgDecimal)(pr) - newSum := new(types.MyDecimal) for _, row := range rowsInGroup { inputSum, isNull, err := e.args[1].EvalDecimal(sctx, row) if err != nil { @@ -111,6 +110,7 @@ func (e *avgPartial4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsIn continue } + newSum := new(types.MyDecimal) err = types.DecimalAdd(&p.sum, inputSum, newSum) if err != nil { return errors.Trace(err) @@ -121,6 +121,21 @@ func (e *avgPartial4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsIn return nil } +func (e *avgPartial4Decimal) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4AvgDecimal)(src), (*partialResult4AvgDecimal)(dst) + if p1.count == 0 { + return nil + } + newSum := new(types.MyDecimal) + err := types.DecimalAdd(&p1.sum, &p2.sum, newSum) + if err != nil { + return errors.Trace(err) + } + p2.sum = *newSum + p2.count += p1.count + return nil +} + type partialResult4AvgDistinctDecimal struct { partialResult4AvgDecimal valSet decimalSet @@ -146,7 +161,6 @@ func (e *avgOriginal4DistinctDecimal) ResetPartialResult(pr PartialResult) { func (e *avgOriginal4DistinctDecimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { p := (*partialResult4AvgDistinctDecimal)(pr) - newSum := new(types.MyDecimal) for _, row := range rowsInGroup { input, isNull, err := e.args[0].EvalDecimal(sctx, row) if err != nil { @@ -156,6 +170,7 @@ func (e *avgOriginal4DistinctDecimal) UpdatePartialResult(sctx sessionctx.Contex continue } + newSum := new(types.MyDecimal) err = types.DecimalAdd(&p.sum, input, newSum) if err != nil { return errors.Trace(err) @@ -261,13 +276,19 @@ func (e *avgPartial4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsIn if isNull { continue } - p.sum += inputSum p.count += inputCount } return nil } +func (e *avgPartial4Float64) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4AvgFloat64)(src), (*partialResult4AvgFloat64)(dst) + p2.sum += p1.sum + p2.count += p1.count + return nil +} + type partialResult4AvgDistinctFloat64 struct { partialResult4AvgFloat64 valSet float64Set diff --git a/executor/aggfuncs/func_avg_test.go b/executor/aggfuncs/func_avg_test.go new file mode 100644 index 0000000000000..4b5b5667910c2 --- /dev/null +++ b/executor/aggfuncs/func_avg_test.go @@ -0,0 +1,347 @@ +// Copyright 2018 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_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/executor/aggfuncs" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +func (s *testSuite) TestMergePartialResult4Count(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendInt64(0, i) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncCount, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeLong), Index: 0}}, + } + finalDesc := desc.Split([]int{0}) + + // build count func for partial phase. + partialCountFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialCountFunc.AllocPartialResult() + + // build final func for final phase. + finalCountFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalCountFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialCountFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + partialCountFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + c.Assert(resultChk.GetRow(0).GetInt64(0), Equals, int64(5)) + + // suppose there are two partial workers. + partialPr2 := partialPr1 + + // merge two partial results. + err := finalCountFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalCountFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalCountFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + c.Assert(resultChk.GetRow(0).GetInt64(0), Equals, int64(10)) +} + +func (s *testSuite) TestMergePartialResult4AvgDecimal(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeNewDecimal)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendMyDecimal(0, types.NewDecFromInt(i)) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncAvg, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong), Index: 0}}, + RetTp: types.NewFieldType(mysql.TypeNewDecimal), + } + finalDesc := desc.Split([]int{0, 1}) + + // build avg func for partial phase. + partialAvgFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialAvgFunc.AllocPartialResult() + partialPr2 := partialAvgFunc.AllocPartialResult() + + // build final func for final phase. + finalAvgFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalAvgFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeNewDecimal)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialAvgFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + // (0+1+2+3+4) / 5 + partialAvgFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromInt(2)) == 0, IsTrue) + + row := iter.Begin() + row = iter.Next() + for row = iter.Next(); row != iter.End(); row = iter.Next() { + partialAvgFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr2) + } + resultChk.Reset() + // (2+3+4) / 3 + partialAvgFunc.AppendFinalResult2Chunk(s.ctx, partialPr2, resultChk) + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromFloatForTest(3)) == 0, IsTrue) + + // merge two partial results. + err := finalAvgFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalAvgFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalAvgFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + // (10 + 9) / 8 + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromFloatForTest(2.375)) == 0, IsTrue) +} + +func (s *testSuite) TestMergePartialResult4AvgFloat(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendFloat64(0, float64(i)) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncAvg, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeDouble), Index: 0}}, + RetTp: types.NewFieldType(mysql.TypeDouble), + } + finalDesc := desc.Split([]int{0, 1}) + + // build avg func for partial phase. + partialAvgFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialAvgFunc.AllocPartialResult() + partialPr2 := partialAvgFunc.AllocPartialResult() + + // build final func for final phase. + finalAvgFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalAvgFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialAvgFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + partialAvgFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + // (0+1+2+3+4) / 5 + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(2), IsTrue) + + row := iter.Begin() + row = iter.Next() + for row = iter.Next(); row != iter.End(); row = iter.Next() { + partialAvgFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr2) + } + resultChk.Reset() + partialAvgFunc.AppendFinalResult2Chunk(s.ctx, partialPr2, resultChk) + // (2+3+4) / 3 + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(3), IsTrue) + + // merge two partial results. + err := finalAvgFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalAvgFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalAvgFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + // (10 + 9) / 8 + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(2.375), IsTrue) +} + +func (s *testSuite) TestMergePartialResult4SumDecimal(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeNewDecimal)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendMyDecimal(0, types.NewDecFromInt(i)) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncSum, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong), Index: 0}}, + RetTp: types.NewFieldType(mysql.TypeNewDecimal), + } + finalDesc := desc.Split([]int{0}) + + // build sum func for partial phase. + partialSumFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialSumFunc.AllocPartialResult() + partialPr2 := partialSumFunc.AllocPartialResult() + + // build final func for final phase. + finalAvgFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalAvgFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeNewDecimal)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialSumFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + // 0+1+2+3+4 + partialSumFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromInt(10)) == 0, IsTrue) + + row := iter.Begin() + row = iter.Next() + for row = iter.Next(); row != iter.End(); row = iter.Next() { + partialSumFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr2) + } + resultChk.Reset() + // 2+3+4 + partialSumFunc.AppendFinalResult2Chunk(s.ctx, partialPr2, resultChk) + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromInt(9)) == 0, IsTrue) + + // merge two partial results. + err := finalAvgFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalAvgFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalAvgFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + // 10+9 + c.Assert(resultChk.GetRow(0).GetMyDecimal(0).Compare(types.NewDecFromInt(19)) == 0, IsTrue) +} + +func (s *testSuite) TestMergePartialResult4SumFloat(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendFloat64(0, float64(i)) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncSum, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeDouble), Index: 0}}, + RetTp: types.NewFieldType(mysql.TypeDouble), + } + finalDesc := desc.Split([]int{0}) + + // build sum func for partial phase. + partialSumFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialSumFunc.AllocPartialResult() + partialPr2 := partialSumFunc.AllocPartialResult() + + // build final func for final phase. + finalAvgFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalAvgFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialSumFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + partialSumFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + // (0+1+2+3+4) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(10), IsTrue) + + row := iter.Begin() + row = iter.Next() + for row = iter.Next(); row != iter.End(); row = iter.Next() { + partialSumFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr2) + } + resultChk.Reset() + partialSumFunc.AppendFinalResult2Chunk(s.ctx, partialPr2, resultChk) + // (2+3+4) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(9), IsTrue) + + // merge two partial results. + err := finalAvgFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalAvgFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalAvgFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + // (10 + 9) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(19), IsTrue) +} + +func (s *testSuite) TestMergePartialResult4MaxFloat(c *C) { + srcChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 5) + for i := int64(0); i < 5; i++ { + srcChk.AppendFloat64(0, float64(i)) + } + iter := chunk.NewIterator4Chunk(srcChk) + + desc := &aggregation.AggFuncDesc{ + Name: ast.AggFuncMax, + Mode: aggregation.CompleteMode, + Args: []expression.Expression{&expression.Column{RetType: types.NewFieldType(mysql.TypeDouble), Index: 0}}, + RetTp: types.NewFieldType(mysql.TypeDouble), + } + finalDesc := desc.Split([]int{0}) + + // build max func for partial phase. + partialMaxFunc := aggfuncs.Build(s.ctx, desc, 0) + partialPr1 := partialMaxFunc.AllocPartialResult() + partialPr2 := partialMaxFunc.AllocPartialResult() + + // build final func for final phase. + finalAvgFunc := aggfuncs.Build(s.ctx, finalDesc, 0) + finalPr := finalAvgFunc.AllocPartialResult() + resultChk := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeDouble)}, 1) + + // update partial result. + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + partialMaxFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr1) + } + partialMaxFunc.AppendFinalResult2Chunk(s.ctx, partialPr1, resultChk) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(4), IsTrue) + + row := iter.Begin() + row = iter.Next() + for row = iter.Next(); row != iter.End(); row = iter.Next() { + partialMaxFunc.UpdatePartialResult(s.ctx, []chunk.Row{row}, partialPr2) + } + resultChk.Reset() + partialMaxFunc.AppendFinalResult2Chunk(s.ctx, partialPr2, resultChk) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(4), IsTrue) + + // merge two partial results. + err := finalAvgFunc.MergePartialResult(s.ctx, partialPr1, finalPr) + c.Assert(err, IsNil) + err = finalAvgFunc.MergePartialResult(s.ctx, partialPr2, finalPr) + c.Assert(err, IsNil) + + resultChk.Reset() + err = finalAvgFunc.AppendFinalResult2Chunk(s.ctx, finalPr, resultChk) + c.Assert(err, IsNil) + c.Assert(resultChk.GetRow(0).GetFloat64(0) == float64(4), IsTrue) +} diff --git a/executor/aggfuncs/func_bitfuncs.go b/executor/aggfuncs/func_bitfuncs.go index 76c8e72d00d8f..d30e27bff3a1b 100644 --- a/executor/aggfuncs/func_bitfuncs.go +++ b/executor/aggfuncs/func_bitfuncs.go @@ -61,6 +61,12 @@ func (e *bitOrUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (*bitOrUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) + *p2 |= uint64(*p1) + return nil +} + type bitXorUint64 struct { baseBitAggFunc } @@ -80,6 +86,12 @@ func (e *bitXorUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return nil } +func (*bitXorUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) + *p2 ^= uint64(*p1) + return nil +} + type bitAndUint64 struct { baseBitAggFunc } @@ -109,3 +121,9 @@ func (e *bitAndUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup } return nil } + +func (*bitAndUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) + *p2 &= uint64(*p1) + return nil +} diff --git a/executor/aggfuncs/func_count.go b/executor/aggfuncs/func_count.go index 0948eeae1bfde..8cc682e91200f 100644 --- a/executor/aggfuncs/func_count.go +++ b/executor/aggfuncs/func_count.go @@ -207,6 +207,12 @@ func (e *countPartial) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return nil } +func (*countPartial) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4Count)(src), (*partialResult4Count)(dst) + *p2 += *p1 + return nil +} + type countOriginalWithDistinct struct { baseCount } diff --git a/executor/aggfuncs/func_first_row.go b/executor/aggfuncs/func_first_row.go index 801883eb46ef9..92d39bc9efc8d 100644 --- a/executor/aggfuncs/func_first_row.go +++ b/executor/aggfuncs/func_first_row.go @@ -107,6 +107,14 @@ func (e *firstRow4Int) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return nil } +func (*firstRow4Int) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowInt)(src), (*partialResult4FirstRowInt)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} + func (e *firstRow4Int) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowInt)(pr) if p.isNull || !p.gotFirstRow { @@ -145,6 +153,13 @@ func (e *firstRow4Float32) UpdatePartialResult(sctx sessionctx.Context, rowsInGr } return nil } +func (*firstRow4Float32) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowFloat32)(src), (*partialResult4FirstRowFloat32)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} func (e *firstRow4Float32) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowFloat32)(pr) @@ -185,6 +200,13 @@ func (e *firstRow4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGr return nil } +func (*firstRow4Float64) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowFloat64)(src), (*partialResult4FirstRowFloat64)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} func (e *firstRow4Float64) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowFloat64)(pr) if p.isNull || !p.gotFirstRow { @@ -224,6 +246,14 @@ func (e *firstRow4String) UpdatePartialResult(sctx sessionctx.Context, rowsInGro return nil } +func (*firstRow4String) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowString)(src), (*partialResult4FirstRowString)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} + func (e *firstRow4String) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowString)(pr) if p.isNull || !p.gotFirstRow { @@ -262,6 +292,14 @@ func (e *firstRow4Time) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup } return nil } +func (*firstRow4Time) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowTime)(src), (*partialResult4FirstRowTime)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil + +} func (e *firstRow4Time) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowTime)(pr) @@ -301,6 +339,13 @@ func (e *firstRow4Duration) UpdatePartialResult(sctx sessionctx.Context, rowsInG } return nil } +func (*firstRow4Duration) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowDuration)(src), (*partialResult4FirstRowDuration)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} func (e *firstRow4Duration) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowDuration)(pr) @@ -340,6 +385,13 @@ func (e *firstRow4JSON) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup } return nil } +func (*firstRow4JSON) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowJSON)(src), (*partialResult4FirstRowJSON)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} func (e *firstRow4JSON) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowJSON)(pr) @@ -392,3 +444,11 @@ func (e *firstRow4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr P chk.AppendMyDecimal(e.ordinal, &p.val) return nil } + +func (*firstRow4Decimal) MergePartialResult(sctx sessionctx.Context, src PartialResult, dst PartialResult) error { + p1, p2 := (*partialResult4FirstRowDecimal)(src), (*partialResult4FirstRowDecimal)(dst) + if !p2.gotFirstRow { + *p2 = *p1 + } + return nil +} diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index ecb22cd418daf..433a9b90a5b06 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -110,6 +110,21 @@ func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (e *groupConcat) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4GroupConcat)(src), (*partialResult4GroupConcat)(dst) + if p1.buffer == nil { + return nil + } + if p2.buffer == nil { + p2.buffer = p1.buffer + return nil + } + p2.buffer.WriteString(e.sep) + p2.buffer.WriteString(p1.buffer.String()) + e.truncatePartialResultIfNeed(sctx, p2.buffer) + return nil +} + type partialResult4GroupConcatDistinct struct { basePartialResult4GroupConcat valsBuf *bytes.Buffer diff --git a/executor/aggfuncs/func_max_min.go b/executor/aggfuncs/func_max_min.go index a795424999461..c15c79e417298 100644 --- a/executor/aggfuncs/func_max_min.go +++ b/executor/aggfuncs/func_max_min.go @@ -124,6 +124,21 @@ func (e *maxMin4Int) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [] return nil } +func (e *maxMin4Int) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinInt)(src), (*partialResult4MaxMinInt)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + if e.isMax && p1.val > p2.val || !e.isMax && p1.val < p2.val { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4Uint struct { baseMaxMinAggFunc } @@ -173,6 +188,21 @@ func (e *maxMin4Uint) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (e *maxMin4Uint) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinUint)(src), (*partialResult4MaxMinUint)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + if e.isMax && p1.val > p2.val || !e.isMax && p1.val < p2.val { + p2.val, p2.isNull = p1.val, false + } + return nil +} + // maxMin4Float32 gets a float32 input and returns a float32 result. type maxMin4Float32 struct { baseMaxMinAggFunc @@ -223,6 +253,21 @@ func (e *maxMin4Float32) UpdatePartialResult(sctx sessionctx.Context, rowsInGrou return nil } +func (e *maxMin4Float32) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinFloat32)(src), (*partialResult4MaxMinFloat32)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + if e.isMax && p1.val > p2.val || !e.isMax && p1.val < p2.val { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4Float64 struct { baseMaxMinAggFunc } @@ -271,6 +316,21 @@ func (e *maxMin4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGrou return nil } +func (e *maxMin4Float64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinFloat64)(src), (*partialResult4MaxMinFloat64)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + if e.isMax && p1.val > p2.val || !e.isMax && p1.val < p2.val { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4Decimal struct { baseMaxMinAggFunc } @@ -319,6 +379,22 @@ func (e *maxMin4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGrou return nil } +func (e *maxMin4Decimal) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinDecimal)(src), (*partialResult4MaxMinDecimal)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + cmp := (&p1.val).Compare(&p2.val) + if e.isMax && cmp > 0 || !e.isMax && cmp < 0 { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4String struct { baseMaxMinAggFunc } @@ -371,6 +447,22 @@ func (e *maxMin4String) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return nil } +func (e *maxMin4String) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinString)(src), (*partialResult4MaxMinString)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + cmp := types.CompareString(p1.val, p2.val) + if e.isMax && cmp > 0 || !e.isMax && cmp < 0 { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4Time struct { baseMaxMinAggFunc } @@ -419,6 +511,22 @@ func (e *maxMin4Time) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (e *maxMin4Time) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4Time)(src), (*partialResult4Time)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + cmp := p1.val.Compare(p2.val) + if e.isMax && cmp == 1 || !e.isMax && cmp == -1 { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4Duration struct { baseMaxMinAggFunc } @@ -467,6 +575,22 @@ func (e *maxMin4Duration) UpdatePartialResult(sctx sessionctx.Context, rowsInGro return nil } +func (e *maxMin4Duration) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinDuration)(src), (*partialResult4MaxMinDuration)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + cmp := p1.val.Compare(p2.val) + if e.isMax && cmp == 1 || !e.isMax && cmp == -1 { + p2.val, p2.isNull = p1.val, false + } + return nil +} + type maxMin4JSON struct { baseMaxMinAggFunc } @@ -508,9 +632,26 @@ func (e *maxMin4JSON) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ continue } cmp := json.CompareBinary(input, p.val) - if e.isMax && cmp > 1 || !e.isMax && cmp < -1 { + if e.isMax && cmp > 0 || !e.isMax && cmp < 0 { p.val = input } } return nil } + +func (e *maxMin4JSON) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4MaxMinJSON)(src), (*partialResult4MaxMinJSON)(dst) + if p1.isNull { + return nil + } + if p2.isNull { + *p2 = *p1 + return nil + } + cmp := json.CompareBinary(p1.val, p2.val) + if e.isMax && cmp > 0 || !e.isMax && cmp < 0 { + p2.val = p1.val + p2.isNull = false + } + return nil +} diff --git a/executor/aggfuncs/func_sum.go b/executor/aggfuncs/func_sum.go index 29ea68c3125e5..21704b61490f7 100644 --- a/executor/aggfuncs/func_sum.go +++ b/executor/aggfuncs/func_sum.go @@ -90,6 +90,16 @@ func (e *sum4Float64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (e *sum4Float64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4SumFloat64)(src), (*partialResult4SumFloat64)(dst) + if p1.isNull { + return nil + } + p2.val += p1.val + p2.isNull = false + return nil +} + type sum4Decimal struct { baseSumAggFunc } @@ -117,7 +127,6 @@ func (e *sum4Decimal) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Partia func (e *sum4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { p := (*partialResult4SumDecimal)(pr) - newSum := new(types.MyDecimal) for _, row := range rowsInGroup { input, isNull, err := e.args[0].EvalDecimal(sctx, row) if err != nil { @@ -132,6 +141,7 @@ func (e *sum4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ continue } + newSum := new(types.MyDecimal) err = types.DecimalAdd(&p.val, input, newSum) if err != nil { return errors.Trace(err) @@ -141,6 +151,21 @@ func (e *sum4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return nil } +func (e *sum4Decimal) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { + p1, p2 := (*partialResult4SumDecimal)(src), (*partialResult4SumDecimal)(dst) + if p1.isNull { + return nil + } + newSum := new(types.MyDecimal) + err := types.DecimalAdd(&p1.val, &p2.val, newSum) + if err != nil { + return errors.Trace(err) + } + p2.val = *newSum + p2.isNull = false + return nil +} + type sum4DistinctFloat64 struct { baseSumAggFunc } @@ -208,7 +233,6 @@ func (e *sum4DistinctDecimal) ResetPartialResult(pr PartialResult) { func (e *sum4DistinctDecimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) error { p := (*partialResult4SumDistinctDecimal)(pr) - var newSum types.MyDecimal for _, row := range rowsInGroup { input, isNull, err := e.args[0].EvalDecimal(sctx, row) if err != nil { @@ -223,10 +247,11 @@ func (e *sum4DistinctDecimal) UpdatePartialResult(sctx sessionctx.Context, rowsI p.isNull = false continue } - if err = types.DecimalAdd(&p.val, input, &newSum); err != nil { + newSum := new(types.MyDecimal) + if err = types.DecimalAdd(&p.val, input, newSum); err != nil { return errors.Trace(err) } - p.val = newSum + p.val = *newSum } return nil } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 5b16781b93b87..e10a2953d6901 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -325,6 +325,11 @@ func (s *testSuite) TestAggregation(c *C) { tk.MustExec("create table t(a int)") tk.MustExec("insert into t value(null)") tk.MustQuery("select group_concat(a), group_concat(distinct a) from t").Check(testkit.Rows(" ")) + + tk.MustExec("drop table t") + tk.MustExec("create table t(a decimal(10, 4))") + tk.MustExec("insert into t value(0), (-0.9871), (-0.9871)") + tk.MustQuery("select sum(a) from (select a from t union all select a from t) tmp").Check(testkit.Rows("-3.9484")) } func (s *testSuite) TestStreamAggPushDown(c *C) { diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index c434dc8dd2995..578c86fc9957a 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -23,6 +23,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -79,6 +80,53 @@ func (a *AggFuncDesc) Clone() *AggFuncDesc { return &clone } +// Split splits `a` into two aggregate descriptors for partial phase and +// final phase individually. +// This function is only used when executing aggregate function parallelly. +// ordinal indicates the column ordinal of the intermediate result. +func (a *AggFuncDesc) Split(ordinal []int) (finalAggDesc *AggFuncDesc) { + if a.Mode == CompleteMode { + a.Mode = Partial1Mode + } else if a.Mode == FinalMode { + a.Mode = Partial2Mode + } else { + return + } + finalAggDesc = &AggFuncDesc{ + Name: a.Name, + Mode: FinalMode, // We only support FinalMode now in final phase. + HasDistinct: a.HasDistinct, + RetTp: a.RetTp, + } + switch a.Name { + case ast.AggFuncAvg: + args := make([]expression.Expression, 0, 2) + args = append(args, &expression.Column{ + ColName: model.NewCIStr(fmt.Sprintf("avg_final_col_%d", ordinal[0])), + Index: ordinal[0], + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + args = append(args, &expression.Column{ + ColName: model.NewCIStr(fmt.Sprintf("avg_final_col_%d", ordinal[1])), + Index: ordinal[1], + RetType: a.RetTp, + }) + finalAggDesc.Args = args + default: + args := make([]expression.Expression, 0, 1) + args = append(args, &expression.Column{ + ColName: model.NewCIStr(fmt.Sprintf("%s_final_col_%d", a.Name, ordinal[0])), + Index: ordinal[0], + RetType: a.RetTp, + }) + finalAggDesc.Args = args + if finalAggDesc.Name == ast.AggFuncGroupConcat { + finalAggDesc.Args = append(finalAggDesc.Args, a.Args[len(a.Args)-1]) // separator + } + } + return finalAggDesc +} + // String implements the fmt.Stringer interface. func (a *AggFuncDesc) String() string { buffer := bytes.NewBufferString(a.Name) diff --git a/types/mydecimal.go b/types/mydecimal.go index 5579979efe16b..5d2c88fbefe7c 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -1409,6 +1409,8 @@ func DecimalNeg(from *MyDecimal) *MyDecimal { } // DecimalAdd adds two decimals, sets the result to 'to'. +// Note: DO NOT use `from1` or `from2` as `to` since the metadata +// of `to` may be changed during evaluating. func DecimalAdd(from1, from2, to *MyDecimal) error { to.resultFrac = myMaxInt8(from1.resultFrac, from2.resultFrac) if from1.negative == from2.negative { From 1d567efccab002f83ede4243a1d3436f1fae809e Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Mon, 13 Aug 2018 15:59:30 +0800 Subject: [PATCH 40/87] executor: fix group_concat when chunk size is set to 1 (#7328) --- executor/aggfuncs/func_group_concat.go | 17 +++++++++++------ executor/aggregate_test.go | 4 ++++ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index 433a9b90a5b06..dc9845f4df997 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -82,9 +82,12 @@ func (e *groupConcat) ResetPartialResult(pr PartialResult) { func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (err error) { p := (*partialResult4GroupConcat)(pr) - v, isNull, isWriteSep := "", false, false + v, isNull := "", false for _, row := range rowsInGroup { - isWriteSep = false + if p.buffer != nil { + p.buffer.WriteString(e.sep) + } + isAllNull := true for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { @@ -93,18 +96,20 @@ func (e *groupConcat) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ if isNull { continue } - isWriteSep = true + isAllNull = false if p.buffer == nil { p.buffer = &bytes.Buffer{} } p.buffer.WriteString(v) } - if isWriteSep { - p.buffer.WriteString(e.sep) + if isAllNull { + if p.buffer != nil { + p.buffer.Truncate(p.buffer.Len() - len(e.sep)) + } + continue } } if p.buffer != nil { - p.buffer.Truncate(p.buffer.Len() - len(e.sep)) return e.truncatePartialResultIfNeed(sctx, p.buffer) } return nil diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index e10a2953d6901..f096dfed3066c 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -275,6 +275,10 @@ func (s *testSuite) TestAggregation(c *C) { tk.MustQuery("select 10 from idx_agg group by b").Check(testkit.Rows("10", "10")) tk.MustQuery("select 11 from idx_agg group by a").Check(testkit.Rows("11", "11")) + tk.MustExec("set @@tidb_max_chunk_size=1;") + tk.MustQuery("select group_concat(b) from idx_agg group by b;").Check(testkit.Rows("1", "2,2")) + tk.MustExec("set @@tidb_max_chunk_size=2;") + tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int(11), b decimal(15,2))") tk.MustExec("insert into t values(1,771.64),(2,378.49),(3,920.92),(4,113.97)") From 0b031145c84da01abbdfd6515b57965aee18682c Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 13 Aug 2018 16:19:09 +0800 Subject: [PATCH 41/87] remove redundency code (#7370) --- plan/rule_column_pruning.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plan/rule_column_pruning.go b/plan/rule_column_pruning.go index e27aef65756e5..8c0cedc22ee6b 100644 --- a/plan/rule_column_pruning.go +++ b/plan/rule_column_pruning.go @@ -121,7 +121,7 @@ func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) { if len(cols) == 0 { ls.ByItems = append(ls.ByItems[:i], ls.ByItems[i+1:]...) } else { - parentUsedCols = append(parentUsedCols, expression.ExtractColumns(ls.ByItems[i].Expr)...) + parentUsedCols = append(parentUsedCols, cols...) } } child.PruneColumns(parentUsedCols) From 87b16fca27d6d772af2f67f9da68ba12348753c8 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Mon, 13 Aug 2018 16:38:36 +0800 Subject: [PATCH 42/87] support for flush status (#7369) --- ast/misc.go | 1 + parser/parser.y | 6 ++++++ parser/parser_test.go | 1 + 3 files changed, 8 insertions(+) diff --git a/ast/misc.go b/ast/misc.go index dec9ab4637957..49c45dceff105 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -339,6 +339,7 @@ const ( FlushNone FlushStmtType = iota FlushTables FlushPrivileges + FlushStatus ) // FlushStmt is a statement to flush tables/privileges/optimizer costs and so on. diff --git a/parser/parser.y b/parser/parser.y index 88156a729dc87..e504f4ef949e6 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -5518,6 +5518,12 @@ FlushOption: Tp: ast.FlushPrivileges, } } +| "STATUS" + { + $$ = &ast.FlushStmt{ + Tp: ast.FlushStatus, + } + } | TableOrTables TableNameListOpt WithReadLockOpt { $$ = &ast.FlushStmt{ diff --git a/parser/parser_test.go b/parser/parser_test.go index 2c3689671ecce..f74536fe8462f 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -618,6 +618,7 @@ func (s *testParserSuite) TestDBAStmt(c *C) { {"flush tables tbl1, tbl2, tbl3", true}, {"flush tables tbl1, tbl2, tbl3 with read lock", true}, {"flush privileges", true}, + {"flush status", true}, } s.RunTest(c, table) } From cf5e392b246f544846f4e66affc70630f486af4d Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 13 Aug 2018 19:03:58 +0800 Subject: [PATCH 43/87] executor: change joinResultGenerator to joiner (#7323) --- executor/builder.go | 32 +-- executor/index_lookup_join.go | 6 +- executor/join.go | 24 +- executor/join_result_generators.go | 426 ----------------------------- executor/joiner.go | 426 +++++++++++++++++++++++++++++ executor/merge_join.go | 12 +- executor/pkg_test.go | 14 +- 7 files changed, 470 insertions(+), 470 deletions(-) delete mode 100644 executor/join_result_generators.go create mode 100644 executor/joiner.go diff --git a/executor/builder.go b/executor/builder.go index eef9a5ad658d6..7122e5f7a33a1 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -714,7 +714,7 @@ func (b *executorBuilder) buildMergeJoin(v *plan.PhysicalMergeJoin) Executor { e := &MergeJoinExec{ stmtCtx: b.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID(), leftExec, rightExec), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, + joiner: newJoiner(b.ctx, v.JoinType, v.JoinType == plan.RightOuterJoin, defaultValues, v.OtherConditions, leftExec.retTypes(), rightExec.retTypes()), } @@ -816,9 +816,9 @@ func (b *executorBuilder) buildHashJoin(v *plan.PhysicalHashJoin) Executor { defaultValues = make([]types.Datum, e.innerExec.Schema().Len()) } } - e.resultGenerators = make([]joinResultGenerator, e.concurrency) + e.joiners = make([]joiner, e.concurrency) for i := uint(0); i < e.concurrency; i++ { - e.resultGenerators[i] = newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, + e.joiners[i] = newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() @@ -1151,7 +1151,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply if defaultValues == nil { defaultValues = make([]types.Datum, v.Children()[v.InnerChildIdx].Schema().Len()) } - generator := newJoinResultGenerator(b.ctx, v.JoinType, v.InnerChildIdx == 0, + tupleJoiner := newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, otherConditions, leftChild.retTypes(), rightChild.retTypes()) outerExec, innerExec := leftChild, rightChild outerFilter, innerFilter := v.LeftConditions, v.RightConditions @@ -1160,14 +1160,14 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply outerFilter, innerFilter = v.RightConditions, v.LeftConditions } e := &NestedLoopApplyExec{ - baseExecutor: newBaseExecutor(b.ctx, apply.Schema(), v.ExplainID(), outerExec, innerExec), - innerExec: innerExec, - outerExec: outerExec, - outerFilter: outerFilter, - innerFilter: innerFilter, - outer: v.JoinType != plan.InnerJoin, - resultGenerator: generator, - outerSchema: apply.OuterSchema, + baseExecutor: newBaseExecutor(b.ctx, apply.Schema(), v.ExplainID(), outerExec, innerExec), + innerExec: innerExec, + outerExec: outerExec, + outerFilter: outerFilter, + innerFilter: innerFilter, + outer: v.JoinType != plan.InnerJoin, + joiner: tupleJoiner, + outerSchema: apply.OuterSchema, } metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec").Inc() return e @@ -1495,10 +1495,10 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plan.PhysicalIndexJoin) Execut readerBuilder: &dataReaderBuilder{innerPlan, b}, rowTypes: innerTypes, }, - workerWg: new(sync.WaitGroup), - resultGenerator: newJoinResultGenerator(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), - indexRanges: v.Ranges, - keyOff2IdxOff: v.KeyOff2IdxOff, + workerWg: new(sync.WaitGroup), + joiner: newJoiner(b.ctx, v.JoinType, v.OuterIndex == 1, defaultValues, v.OtherConditions, leftTypes, rightTypes), + indexRanges: v.Ranges, + keyOff2IdxOff: v.KeyOff2IdxOff, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a68d69aa66ec4..92378d80a1e07 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -60,7 +60,7 @@ type IndexLookUpJoin struct { joinResult *chunk.Chunk innerIter chunk.Iterator - resultGenerator joinResultGenerator + joiner joiner indexRanges []*ranger.Range keyOff2IdxOff []int @@ -207,7 +207,7 @@ func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { outerRow := task.outerResult.GetRow(task.cursor) if e.innerIter.Current() != e.innerIter.End() { - matched, err := e.resultGenerator.tryToMatch(outerRow, e.innerIter, chk) + matched, err := e.joiner.tryToMatch(outerRow, e.innerIter, chk) if err != nil { return errors.Trace(err) } @@ -215,7 +215,7 @@ func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { } if e.innerIter.Current() == e.innerIter.End() { if !task.hasMatch { - e.resultGenerator.onMissMatch(outerRow, chk) + e.joiner.onMissMatch(outerRow, chk) } task.cursor++ task.hasMatch = false diff --git a/executor/join.go b/executor/join.go index f7e8eba8c8e57..8eb1b953c5691 100644 --- a/executor/join.go +++ b/executor/join.go @@ -58,9 +58,9 @@ type HashJoinExec struct { joinType plan.JoinType innerIdx int - // We build individual resultGenerator for each join worker when use chunk-based execution, - // to avoid the concurrency of joinResultGenerator.chk and joinResultGenerator.selected. - resultGenerators []joinResultGenerator + // We build individual joiner for each join worker when use chunk-based execution, + // to avoid the concurrency of joiner.chk and joiner.selected. + joiners []joiner outerKeyColIdx []int innerKeyColIdx []int @@ -385,13 +385,13 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R return false, joinResult } if hasNull { - e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + e.joiners[workerID].onMissMatch(outerRow, joinResult.chk) return true, joinResult } e.hashTableValBufs[workerID] = e.hashTable.Get(joinKey, e.hashTableValBufs[workerID][:0]) innerPtrs := e.hashTableValBufs[workerID] if len(innerPtrs) == 0 { - e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + e.joiners[workerID].onMissMatch(outerRow, joinResult.chk) return true, joinResult } innerRows := make([]chunk.Row, 0, len(innerPtrs)) @@ -403,7 +403,7 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R iter := chunk.NewIterator4Slice(innerRows) hasMatch := false for iter.Begin(); iter.Current() != iter.End(); { - matched, err := e.resultGenerators[workerID].tryToMatch(outerRow, iter, joinResult.chk) + matched, err := e.joiners[workerID].tryToMatch(outerRow, iter, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) return false, joinResult @@ -420,7 +420,7 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R } } if !hasMatch { - e.resultGenerators[workerID].onMissMatch(outerRow, joinResult.chk) + e.joiners[workerID].onMissMatch(outerRow, joinResult.chk) } return true, joinResult } @@ -448,7 +448,7 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu } for i := range selected { if !selected[i] { // process unmatched outer rows - e.resultGenerators[workerID].onMissMatch(outerChk.GetRow(i), joinResult.chk) + e.joiners[workerID].onMissMatch(outerChk.GetRow(i), joinResult.chk) } else { // process matched outer rows ok, joinResult = e.joinMatchedOuterRow2Chunk(workerID, outerChk.GetRow(i), joinResult) if !ok { @@ -562,7 +562,7 @@ type NestedLoopApplyExec struct { outerFilter expression.CNFExprs outer bool - resultGenerator joinResultGenerator + joiner joiner outerSchema []*expression.CorrelatedColumn @@ -632,7 +632,7 @@ func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *ch if selected { return &outerRow, nil } else if e.outer { - e.resultGenerator.onMissMatch(outerRow, chk) + e.joiner.onMissMatch(outerRow, chk) if chk.NumRows() == e.maxChunkSize { return nil, nil } @@ -676,7 +676,7 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { if e.outerRow != nil && !e.hasMatch { - e.resultGenerator.onMissMatch(*e.outerRow, chk) + e.joiner.onMissMatch(*e.outerRow, chk) } e.outerRow, err = e.fetchSelectedOuterRow(ctx, chk) if e.outerRow == nil || err != nil { @@ -695,7 +695,7 @@ func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err e e.innerIter.Begin() } - matched, err := e.resultGenerator.tryToMatch(*e.outerRow, e.innerIter, chk) + matched, err := e.joiner.tryToMatch(*e.outerRow, e.innerIter, chk) e.hasMatch = e.hasMatch || matched if err != nil || chk.NumRows() == e.maxChunkSize { diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go deleted file mode 100644 index 9098502bc2825..0000000000000 --- a/executor/join_result_generators.go +++ /dev/null @@ -1,426 +0,0 @@ -// Copyright 2017 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 executor - -import ( - "github.com/juju/errors" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/plan" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" -) - -var ( - _ joinResultGenerator = &semiJoinResultGenerator{} - _ joinResultGenerator = &antiSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &antiLeftOuterSemiJoinResultGenerator{} - _ joinResultGenerator = &leftOuterJoinResultGenerator{} - _ joinResultGenerator = &rightOuterJoinResultGenerator{} - _ joinResultGenerator = &innerJoinResultGenerator{} -) - -// joinResultGenerator is used to generate join results according to the join -// type. A typical instruction flow is: -// -// hasMatch := false -// for innerIter.Current() != innerIter.End() { -// matched, err := g.tryToMatch(outer, innerIter, chk) -// // handle err -// hasMatch = hasMatch || matched -// } -// if !hasMatch { -// g.onMissMatch(outer) -// } -// -// NOTE: This interface is **not** thread-safe. -type joinResultGenerator interface { - // tryToMatch tries to join an outer row with a batch of inner rows. When - // 'inners.Len != 0' but all the joined rows are filtered, the outer row is - // considered unmatched. Otherwise, the outer row is matched and some joined - // rows are appended to `chk`. The size of `chk` is limited to MaxChunkSize. - // - // NOTE: Callers need to call this function multiple times to consume all - // the inner rows for an outer row, and dicide whether the outer row can be - // matched with at lease one inner row. - tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) - - // onMissMatch operates on the unmatched outer row according to the join - // type. An outer row can be considered miss matched if: - // 1. it can not pass the filter on the outer table side. - // 2. there is no inner row with the same join key. - // 3. all the joined rows can not pass the filter on the join result. - // - // On these conditions, the caller calls this function to handle the - // unmatched outer rows according to the current join type: - // 1. 'SemiJoin': ignores the unmatched outer row. - // 2. 'AntiSemiJoin': appends the unmatched outer row to the result buffer. - // 3. 'LeftOuterSemiJoin': concats the unmatched outer row with 0 and - // appends it to the result buffer. - // 4. 'AntiLeftOuterSemiJoin': concats the unmatched outer row with 0 and - // appends it to the result buffer. - // 5. 'LeftOuterJoin': concats the unmatched outer row with a row of NULLs - // and appends it to the result buffer. - // 6. 'RightOuterJoin': concats the unmatched outer row with a row of NULLs - // and appends it to the result buffer. - // 7. 'InnerJoin': ignores the unmatched outer row. - onMissMatch(outer chunk.Row, chk *chunk.Chunk) -} - -func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, - outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, - lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { - base := baseJoinResultGenerator{ - ctx: ctx, - conditions: filter, - outerIsRight: outerIsRight, - maxChunkSize: ctx.GetSessionVars().MaxChunkSize, - } - colTypes := make([]*types.FieldType, 0, len(lhsColTypes)+len(rhsColTypes)) - colTypes = append(colTypes, lhsColTypes...) - colTypes = append(colTypes, rhsColTypes...) - base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) - base.selected = make([]bool, 0, chunk.InitialCapacity) - if joinType == plan.LeftOuterJoin || joinType == plan.RightOuterJoin { - innerColTypes := lhsColTypes - if !outerIsRight { - innerColTypes = rhsColTypes - } - base.initDefaultInner(innerColTypes, defaultInner) - } - switch joinType { - case plan.SemiJoin: - return &semiJoinResultGenerator{base} - case plan.AntiSemiJoin: - return &antiSemiJoinResultGenerator{base} - case plan.LeftOuterSemiJoin: - return &leftOuterSemiJoinResultGenerator{base} - case plan.AntiLeftOuterSemiJoin: - return &antiLeftOuterSemiJoinResultGenerator{base} - case plan.LeftOuterJoin: - return &leftOuterJoinResultGenerator{base} - case plan.RightOuterJoin: - return &rightOuterJoinResultGenerator{base} - case plan.InnerJoin: - return &innerJoinResultGenerator{base} - } - panic("unsupported join type in func newJoinResultGenerator()") -} - -type baseJoinResultGenerator struct { - ctx sessionctx.Context - conditions []expression.Expression - defaultInner chunk.Row - outerIsRight bool - chk *chunk.Chunk - selected []bool - maxChunkSize int -} - -func (outputer *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { - mutableRow := chunk.MutRowFromTypes(innerTypes) - mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) - outputer.defaultInner = mutableRow.ToRow() -} - -func (outputer *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { - // Call AppendRow() first to increment the virtual rows. - // Fix: https://github.com/pingcap/tidb/issues/5771 - chk.AppendRow(lhs) - chk.AppendPartialRow(lhs.Len(), rhs) -} - -func (outputer *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { - outputer.selected, err = expression.VectorizedFilter(outputer.ctx, outputer.conditions, chunk.NewIterator4Chunk(input), outputer.selected) - if err != nil { - return false, errors.Trace(err) - } - for i := 0; i < len(outputer.selected); i++ { - if !outputer.selected[i] { - continue - } - matched = true - output.AppendRow(input.GetRow(i)) - } - return matched, nil -} - -type semiJoinResultGenerator struct { - baseJoinResultGenerator -} - -func (outputer *semiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { - if inners.Len() == 0 { - return false, nil - } - - if len(outputer.conditions) == 0 { - chk.AppendPartialRow(0, outer) - inners.ReachEnd() - return true, nil - } - - for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(outputer.chk, inner, outer) - } else { - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - } - - matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - chk.AppendPartialRow(0, outer) - inners.ReachEnd() - return true, nil - } - } - return false, nil -} - -func (outputer *semiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { -} - -type antiSemiJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { - if inners.Len() == 0 { - return false, nil - } - - if len(outputer.conditions) == 0 { - inners.ReachEnd() - return true, nil - } - - for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(outputer.chk, inner, outer) - } else { - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - } - - matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - inners.ReachEnd() - return true, nil - } - } - return false, nil -} - -func (outputer *antiSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendRow(outer) -} - -type leftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { - if inners.Len() == 0 { - return false, nil - } - - if len(outputer.conditions) == 0 { - outputer.onMatch(outer, chk) - inners.ReachEnd() - return true, nil - } - - for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - - matched, err = expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - if err != nil { - return false, errors.Trace(err) - } - if matched { - outputer.onMatch(outer, chk) - inners.ReachEnd() - return true, nil - } - } - return false, nil -} - -func (outputer *leftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) -} - -func (outputer *leftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) -} - -type antiLeftOuterSemiJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { - if inners.Len() == 0 { - return false, nil - } - - if len(outputer.conditions) == 0 { - outputer.onMatch(outer, chk) - inners.ReachEnd() - return true, nil - } - - for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - outputer.chk.Reset() - outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) - - if err != nil { - return false, errors.Trace(err) - } - if matched { - outputer.onMatch(outer, chk) - inners.ReachEnd() - return true, nil - } - } - return false, nil -} - -func (outputer *antiLeftOuterSemiJoinResultGenerator) onMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 0) -} - -func (outputer *antiLeftOuterSemiJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outer) - chk.AppendInt64(outer.Len(), 1) -} - -type leftOuterJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { - if inners.Len() == 0 { - return false, nil - } - - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { - chkForJoin = chk - } - - numToAppend := outputer.maxChunkSize - chk.NumRows() - for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) - inners.Next() - } - if len(outputer.conditions) == 0 { - return true, nil - } - - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) - return matched, errors.Trace(err) -} - -func (outputer *leftOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultInner) -} - -type rightOuterJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { - if inners.Len() == 0 { - return false, nil - } - - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { - chkForJoin = chk - } - - numToAppend := outputer.maxChunkSize - chk.NumRows() - for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { - outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) - inners.Next() - } - if len(outputer.conditions) == 0 { - return true, nil - } - - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) - return matched, errors.Trace(err) -} - -func (outputer *rightOuterJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { - chk.AppendPartialRow(0, outputer.defaultInner) - chk.AppendPartialRow(outputer.defaultInner.Len(), outer) -} - -type innerJoinResultGenerator struct { - baseJoinResultGenerator -} - -// tryToMatch implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { - if inners.Len() == 0 { - return false, nil - } - outputer.chk.Reset() - chkForJoin := outputer.chk - if len(outputer.conditions) == 0 { - chkForJoin = chk - } - inner, numToAppend := inners.Current(), outputer.maxChunkSize-chk.NumRows() - for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { - if outputer.outerIsRight { - outputer.makeJoinRowToChunk(chkForJoin, inner, outer) - } else { - outputer.makeJoinRowToChunk(chkForJoin, outer, inner) - } - } - if len(outputer.conditions) == 0 { - return true, nil - } - - // reach here, chkForJoin is outputer.chk - matched, err := outputer.filter(chkForJoin, chk) - return matched, errors.Trace(err) -} - -func (outputer *innerJoinResultGenerator) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { -} diff --git a/executor/joiner.go b/executor/joiner.go new file mode 100644 index 0000000000000..562afa75dc6be --- /dev/null +++ b/executor/joiner.go @@ -0,0 +1,426 @@ +// Copyright 2017 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 executor + +import ( + "github.com/juju/errors" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/plan" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +var ( + _ joiner = &semiJoiner{} + _ joiner = &antiSemiJoiner{} + _ joiner = &leftOuterSemiJoiner{} + _ joiner = &antiLeftOuterSemiJoiner{} + _ joiner = &leftOuterJoiner{} + _ joiner = &rightOuterJoiner{} + _ joiner = &innerJoiner{} +) + +// joiner is used to generate join results according to the join type. +// A typical instruction flow is: +// +// hasMatch := false +// for innerIter.Current() != innerIter.End() { +// matched, err := j.tryToMatch(outer, innerIter, chk) +// // handle err +// hasMatch = hasMatch || matched +// } +// if !hasMatch { +// j.onMissMatch(outer) +// } +// +// NOTE: This interface is **not** thread-safe. +type joiner interface { + // tryToMatch tries to join an outer row with a batch of inner rows. When + // 'inners.Len != 0' but all the joined rows are filtered, the outer row is + // considered unmatched. Otherwise, the outer row is matched and some joined + // rows are appended to `chk`. The size of `chk` is limited to MaxChunkSize. + // + // NOTE: Callers need to call this function multiple times to consume all + // the inner rows for an outer row, and dicide whether the outer row can be + // matched with at lease one inner row. + tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) + + // onMissMatch operates on the unmatched outer row according to the join + // type. An outer row can be considered miss matched if: + // 1. it can not pass the filter on the outer table side. + // 2. there is no inner row with the same join key. + // 3. all the joined rows can not pass the filter on the join result. + // + // On these conditions, the caller calls this function to handle the + // unmatched outer rows according to the current join type: + // 1. 'SemiJoin': ignores the unmatched outer row. + // 2. 'AntiSemiJoin': appends the unmatched outer row to the result buffer. + // 3. 'LeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 4. 'AntiLeftOuterSemiJoin': concats the unmatched outer row with 0 and + // appends it to the result buffer. + // 5. 'LeftOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. + // 6. 'RightOuterJoin': concats the unmatched outer row with a row of NULLs + // and appends it to the result buffer. + // 7. 'InnerJoin': ignores the unmatched outer row. + onMissMatch(outer chunk.Row, chk *chunk.Chunk) +} + +func newJoiner(ctx sessionctx.Context, joinType plan.JoinType, + outerIsRight bool, defaultInner []types.Datum, filter []expression.Expression, + lhsColTypes, rhsColTypes []*types.FieldType) joiner { + base := baseJoiner{ + ctx: ctx, + conditions: filter, + outerIsRight: outerIsRight, + maxChunkSize: ctx.GetSessionVars().MaxChunkSize, + } + colTypes := make([]*types.FieldType, 0, len(lhsColTypes)+len(rhsColTypes)) + colTypes = append(colTypes, lhsColTypes...) + colTypes = append(colTypes, rhsColTypes...) + base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) + base.selected = make([]bool, 0, chunk.InitialCapacity) + if joinType == plan.LeftOuterJoin || joinType == plan.RightOuterJoin { + innerColTypes := lhsColTypes + if !outerIsRight { + innerColTypes = rhsColTypes + } + base.initDefaultInner(innerColTypes, defaultInner) + } + switch joinType { + case plan.SemiJoin: + return &semiJoiner{base} + case plan.AntiSemiJoin: + return &antiSemiJoiner{base} + case plan.LeftOuterSemiJoin: + return &leftOuterSemiJoiner{base} + case plan.AntiLeftOuterSemiJoin: + return &antiLeftOuterSemiJoiner{base} + case plan.LeftOuterJoin: + return &leftOuterJoiner{base} + case plan.RightOuterJoin: + return &rightOuterJoiner{base} + case plan.InnerJoin: + return &innerJoiner{base} + } + panic("unsupported join type in func newJoiner()") +} + +type baseJoiner struct { + ctx sessionctx.Context + conditions []expression.Expression + defaultInner chunk.Row + outerIsRight bool + chk *chunk.Chunk + selected []bool + maxChunkSize int +} + +func (j *baseJoiner) initDefaultInner(innerTypes []*types.FieldType, defaultInner []types.Datum) { + mutableRow := chunk.MutRowFromTypes(innerTypes) + mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) + j.defaultInner = mutableRow.ToRow() +} + +func (j *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { + // Call AppendRow() first to increment the virtual rows. + // Fix: https://github.com/pingcap/tidb/issues/5771 + chk.AppendRow(lhs) + chk.AppendPartialRow(lhs.Len(), rhs) +} + +func (j *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { + j.selected, err = expression.VectorizedFilter(j.ctx, j.conditions, chunk.NewIterator4Chunk(input), j.selected) + if err != nil { + return false, errors.Trace(err) + } + for i := 0; i < len(j.selected); i++ { + if !j.selected[i] { + continue + } + matched = true + output.AppendRow(input.GetRow(i)) + } + return matched, nil +} + +type semiJoiner struct { + baseJoiner +} + +func (j *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil + } + + if len(j.conditions) == 0 { + chk.AppendPartialRow(0, outer) + inners.ReachEnd() + return true, nil + } + + for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { + j.chk.Reset() + if j.outerIsRight { + j.makeJoinRowToChunk(j.chk, inner, outer) + } else { + j.makeJoinRowToChunk(j.chk, outer, inner) + } + + matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + if err != nil { + return false, errors.Trace(err) + } + if matched { + chk.AppendPartialRow(0, outer) + inners.ReachEnd() + return true, nil + } + } + return false, nil +} + +func (j *semiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +} + +type antiSemiJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil + } + + if len(j.conditions) == 0 { + inners.ReachEnd() + return true, nil + } + + for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { + j.chk.Reset() + if j.outerIsRight { + j.makeJoinRowToChunk(j.chk, inner, outer) + } else { + j.makeJoinRowToChunk(j.chk, outer, inner) + } + + matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + if err != nil { + return false, errors.Trace(err) + } + if matched { + inners.ReachEnd() + return true, nil + } + } + return false, nil +} + +func (j *antiSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendRow(outer) +} + +type leftOuterSemiJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil + } + + if len(j.conditions) == 0 { + j.onMatch(outer, chk) + inners.ReachEnd() + return true, nil + } + + for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { + j.chk.Reset() + j.makeJoinRowToChunk(j.chk, outer, inner) + + matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + if err != nil { + return false, errors.Trace(err) + } + if matched { + j.onMatch(outer, chk) + inners.ReachEnd() + return true, nil + } + } + return false, nil +} + +func (j *leftOuterSemiJoiner) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 1) +} + +func (j *leftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 0) +} + +type antiLeftOuterSemiJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (matched bool, err error) { + if inners.Len() == 0 { + return false, nil + } + + if len(j.conditions) == 0 { + j.onMatch(outer, chk) + inners.ReachEnd() + return true, nil + } + + for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { + j.chk.Reset() + j.makeJoinRowToChunk(j.chk, outer, inner) + matched, err := expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + + if err != nil { + return false, errors.Trace(err) + } + if matched { + j.onMatch(outer, chk) + inners.ReachEnd() + return true, nil + } + } + return false, nil +} + +func (j *antiLeftOuterSemiJoiner) onMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 0) +} + +func (j *antiLeftOuterSemiJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendInt64(outer.Len(), 1) +} + +type leftOuterJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil + } + + j.chk.Reset() + chkForJoin := j.chk + if len(j.conditions) == 0 { + chkForJoin = chk + } + + numToAppend := j.maxChunkSize - chk.NumRows() + for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { + j.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) + inners.Next() + } + if len(j.conditions) == 0 { + return true, nil + } + + // reach here, chkForJoin is j.chk + matched, err := j.filter(chkForJoin, chk) + return matched, errors.Trace(err) +} + +func (j *leftOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, outer) + chk.AppendPartialRow(outer.Len(), j.defaultInner) +} + +type rightOuterJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *rightOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil + } + + j.chk.Reset() + chkForJoin := j.chk + if len(j.conditions) == 0 { + chkForJoin = chk + } + + numToAppend := j.maxChunkSize - chk.NumRows() + for ; inners.Current() != inners.End() && numToAppend > 0; numToAppend-- { + j.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) + inners.Next() + } + if len(j.conditions) == 0 { + return true, nil + } + + // reach here, chkForJoin is j.chk + matched, err := j.filter(chkForJoin, chk) + return matched, errors.Trace(err) +} + +func (j *rightOuterJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { + chk.AppendPartialRow(0, j.defaultInner) + chk.AppendPartialRow(j.defaultInner.Len(), outer) +} + +type innerJoiner struct { + baseJoiner +} + +// tryToMatch implements joiner interface. +func (j *innerJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) (bool, error) { + if inners.Len() == 0 { + return false, nil + } + j.chk.Reset() + chkForJoin := j.chk + if len(j.conditions) == 0 { + chkForJoin = chk + } + inner, numToAppend := inners.Current(), j.maxChunkSize-chk.NumRows() + for ; inner != inners.End() && numToAppend > 0; inner, numToAppend = inners.Next(), numToAppend-1 { + if j.outerIsRight { + j.makeJoinRowToChunk(chkForJoin, inner, outer) + } else { + j.makeJoinRowToChunk(chkForJoin, outer, inner) + } + } + if len(j.conditions) == 0 { + return true, nil + } + + // reach here, chkForJoin is j.chk + matched, err := j.filter(chkForJoin, chk) + return matched, errors.Trace(err) +} + +func (j *innerJoiner) onMissMatch(outer chunk.Row, chk *chunk.Chunk) { +} diff --git a/executor/merge_join.go b/executor/merge_join.go index 1d23b19ef2274..893b29529267c 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -32,9 +32,9 @@ import ( type MergeJoinExec struct { baseExecutor - stmtCtx *stmtctx.StatementContext - compareFuncs []chunk.CompareFunc - resultGenerator joinResultGenerator + stmtCtx *stmtctx.StatementContext + compareFuncs []chunk.CompareFunc + joiner joiner prepared bool outerIdx int @@ -299,7 +299,7 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM } if cmpResult < 0 { - e.resultGenerator.onMissMatch(e.outerTable.row, chk) + e.joiner.onMissMatch(e.outerTable.row, chk) if err != nil { return false, errors.Trace(err) } @@ -313,7 +313,7 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM continue } - matched, err := e.resultGenerator.tryToMatch(e.outerTable.row, e.innerIter4Row, chk) + matched, err := e.joiner.tryToMatch(e.outerTable.row, e.innerIter4Row, chk) if err != nil { return false, errors.Trace(err) } @@ -321,7 +321,7 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM if e.innerIter4Row.Current() == e.innerIter4Row.End() { if !e.outerTable.hasMatch { - e.resultGenerator.onMissMatch(e.outerTable.row, chk) + e.joiner.onMissMatch(e.outerTable.row, chk) } e.outerTable.row = e.outerTable.iter.Next() e.innerIter4Row.Begin() diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 40d4c5670291e..a7438746ad73c 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -80,16 +80,16 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerFilter := expression.NewFunctionInternal(sctx, ast.LT, types.NewFieldType(mysql.TypeTiny), col0, con) innerFilter := outerFilter.Clone() otherFilter := expression.NewFunctionInternal(sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), col0, col1) - generator := newJoinResultGenerator(sctx, plan.InnerJoin, false, + joiner := newJoiner(sctx, plan.InnerJoin, false, make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ - baseExecutor: newBaseExecutor(sctx, joinSchema, ""), - outerExec: outerExec, - innerExec: innerExec, - outerFilter: []expression.Expression{outerFilter}, - innerFilter: []expression.Expression{innerFilter}, - resultGenerator: generator, + baseExecutor: newBaseExecutor(sctx, joinSchema, ""), + outerExec: outerExec, + innerExec: innerExec, + outerFilter: []expression.Expression{outerFilter}, + innerFilter: []expression.Expression{innerFilter}, + joiner: joiner, } join.innerList = chunk.NewList(innerExec.retTypes(), innerExec.maxChunkSize) join.innerChunk = innerExec.newChunk() From 396b2427441c2b9813fcf224ca48436253d4bb00 Mon Sep 17 00:00:00 2001 From: Zhexuan Yang Date: Mon, 13 Aug 2018 19:15:52 +0800 Subject: [PATCH 44/87] store: no need to assign timezone name to dagReq again (#7368) --- store/mockstore/mocktikv/cop_handler_dag.go | 1 - 1 file changed, 1 deletion(-) diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index ab6fa5ca8fc46..0e40eb6211b26 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -153,7 +153,6 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex if sc.TimeZone, err = LocCache.getLoc(dagReq.TimeZoneName); err != nil { return nil, nil, nil, errors.Trace(err) } - dagReq.TimeZoneName = sc.TimeZone.String() } else { sc.TimeZone = time.FixedZone("UTC", int(dagReq.TimeZoneOffset)) } From e43e548a3472671eef89f44a28634ba01732042d Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Mon, 13 Aug 2018 19:35:36 +0800 Subject: [PATCH 45/87] infoschema: fix bug of column size of set and enum type (#7347) --- infoschema/tables.go | 22 ++++++++++++++++++++++ infoschema/tables_test.go | 10 ++++++++++ 2 files changed, 32 insertions(+) diff --git a/infoschema/tables.go b/infoschema/tables.go index cb6fa7bc4d491..02346da318ec8 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -797,6 +797,28 @@ func dataForColumnsInTable(schema *model.DBInfo, tbl *model.TableInfo) [][]types if colLen == types.UnspecifiedLength { colLen = defaultFlen } + if col.Tp == mysql.TypeSet { + // Example: In MySQL set('a','bc','def','ghij') has length 13, because + // len('a')+len('bc')+len('def')+len('ghij')+len(ThreeComma)=13 + // Reference link: https://bugs.mysql.com/bug.php?id=22613 + colLen = 0 + for _, ele := range col.Elems { + colLen += len(ele) + } + if len(col.Elems) != 0 { + colLen += (len(col.Elems) - 1) + } + } else if col.Tp == mysql.TypeEnum { + // Example: In MySQL enum('a', 'ab', 'cdef') has length 4, because + // the longest string in the enum is 'cdef' + // Reference link: https://bugs.mysql.com/bug.php?id=22613 + colLen = 0 + for _, ele := range col.Elems { + if len(ele) > colLen { + colLen = len(ele) + } + } + } if decimal == types.UnspecifiedLength { decimal = defaultDecimal } diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 90dd34a0292a7..6db86406e8a53 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -77,6 +77,16 @@ func (s *testSuite) TestDataForTableRowsCountField(c *C) { tk.MustExec("create user xxx") tk.MustExec("flush privileges") + // Test for length of enum and set + tk.MustExec("drop table if exists t") + tk.MustExec("create table t ( s set('a','bc','def','ghij') default NULL, e1 enum('a', 'ab', 'cdef'), s2 SET('1','2','3','4','1585','ONE','TWO','Y','N','THREE'))") + tk.MustQuery("select column_name, character_maximum_length from information_schema.columns where table_schema=Database() and table_name = 't' and column_name = 's'").Check( + testkit.Rows("s 13")) + tk.MustQuery("select column_name, character_maximum_length from information_schema.columns where table_schema=Database() and table_name = 't' and column_name = 's2'").Check( + testkit.Rows("s2 30")) + tk.MustQuery("select column_name, character_maximum_length from information_schema.columns where table_schema=Database() and table_name = 't' and column_name = 'e1'").Check( + testkit.Rows("e1 4")) + tk1 := testkit.NewTestKit(c, store) tk1.MustExec("use test") c.Assert(tk1.Se.Auth(&auth.UserIdentity{ From 97ae3b336facd6246c56380c607fd0ffbca4dea3 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 14 Aug 2018 00:10:57 +0800 Subject: [PATCH 46/87] sessionctx/binloginfo: add a test for partitioned table (#7381) --- sessionctx/binloginfo/binloginfo_test.go | 26 +++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 5c4baba712b99..dee95184983e5 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -377,7 +377,7 @@ func (s *testBinlogSuite) TestZIgnoreError(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.Se.GetSessionVars().BinlogClient = s.client - tk.MustExec("drop table if exists ignore_error") + tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int)") binloginfo.SetIgnoreError(true) @@ -395,3 +395,27 @@ func (s *testBinlogSuite) TestZIgnoreError(c *C) { binloginfo.DisableSkipBinlogFlag() binloginfo.SetIgnoreError(false) } + +func (s *testBinlogSuite) TestPartitionedTable(c *C) { + // This test checks partitioned table write binlog with table ID, rather than partition ID. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.Se.GetSessionVars().BinlogClient = s.client + tk.MustExec("set @@session.tidb_enable_table_partition=1") + tk.MustExec("drop table if exists t") + tk.MustExec(`create table t (id int) partition by range (id) ( + partition p0 values less than (1), + partition p1 values less than (4), + partition p2 values less than (7), + partition p3 values less than (10))`) + tids := make([]int64, 0, 10) + for i := 0; i < 10; i++ { + tk.MustExec("insert into t values (?)", i) + prewriteVal := getLatestBinlogPrewriteValue(c, s.pump) + tids = append(tids, prewriteVal.Mutations[0].TableId) + } + c.Assert(len(tids), Equals, 10) + for i := 1; i < 10; i++ { + c.Assert(tids[i], Equals, tids[0]) + } +} From bfceb5072f6f7e8de8f3718920300e382405e308 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Tue, 14 Aug 2018 01:14:38 +0800 Subject: [PATCH 47/87] support of character in create table statement (#7378) --- parser/parser.y | 11 +++++++++++ parser/parser_test.go | 6 ++++++ 2 files changed, 17 insertions(+) diff --git a/parser/parser.y b/parser/parser.y index e504f4ef949e6..80b283c5a9a53 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -6094,6 +6094,17 @@ StringType: } $$ = x } +| "NATIONAL" "CHARACTER" FieldLen OptBinary OptCollate + { + x := types.NewFieldType(mysql.TypeString) + x.Flen = $3.(int) + x.Charset = $4.(*ast.OptBinary).Charset + x.Collate = $5.(string) + if $4.(*ast.OptBinary).IsBinary { + x.Flag |= mysql.BinaryFlag + } + $$ = x + } | Varchar FieldLen OptBinary OptCollate { x := types.NewFieldType(mysql.TypeVarchar) diff --git a/parser/parser_test.go b/parser/parser_test.go index f74536fe8462f..6ec18688bf07d 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -201,6 +201,12 @@ func (s *testParserSuite) TestSimple(c *C) { _, err = parser.ParseOneStmt(src, "", "") c.Assert(err, IsNil) + // for #7371, support NATIONAL CHARACTER + // reference link: https://dev.mysql.com/doc/refman/5.7/en/charset-national.html + src = "CREATE TABLE t(c1 NATIONAL CHARACTER(10));" + _, err = parser.ParseOneStmt(src, "", "") + c.Assert(err, IsNil) + src = `CREATE TABLE t(a tinyint signed, b smallint signed, c mediumint signed, From 9667eca8e4ebd7e4fa0dbf98da833c6cd45c11fa Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Tue, 14 Aug 2018 12:44:07 +0800 Subject: [PATCH 48/87] executor: MaxOneRow operator should keep its promise (#7375) --- executor/executor.go | 12 +++++++++--- executor/executor_test.go | 20 ++++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 45578f21e6bad..0a5cf7c39d96b 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -899,11 +899,17 @@ func (e *MaxOneRowExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.AppendNull(i) } return nil - } else if num == 1 { - return nil + } else if num != 1 { + return errors.New("subquery returns more than 1 row") + } + + childChunk := e.children[0].newChunk() + err = e.children[0].Next(ctx, childChunk) + if childChunk.NumRows() != 0 { + return errors.New("subquery returns more than 1 row") } - return errors.New("subquery returns more than 1 row") + return nil } // UnionExec pulls all it's children's result and returns to its parent directly. diff --git a/executor/executor_test.go b/executor/executor_test.go index 0fae51a6ccc6b..e2e53470e550e 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3068,3 +3068,23 @@ func (s *testSuite) TestUpdateJoin(c *C) { tk.MustQuery("select k, v from t5").Check(testkit.Rows("0 0")) } + +func (s *testSuite) TestMaxOneRow(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t1`) + tk.MustExec(`drop table if exists t2`) + tk.MustExec(`create table t1(a double, b double);`) + tk.MustExec(`create table t2(a double, b double);`) + tk.MustExec(`insert into t1 values(1, 1), (2, 2), (3, 3);`) + tk.MustExec(`insert into t2 values(0, 0);`) + tk.MustExec(`set @@tidb_max_chunk_size=1;`) + rs, err := tk.Exec(`select (select t1.a from t1 where t1.a > t2.a) as a from t2;`) + c.Assert(err, IsNil) + + err = rs.Next(context.TODO(), rs.NewChunk()) + c.Assert(err.Error(), Equals, "subquery returns more than 1 row") + + err = rs.Close() + c.Assert(err, IsNil) +} From fc89a6421a4c3ace8624ba17ac127e958d422732 Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 14 Aug 2018 13:20:01 +0800 Subject: [PATCH 49/87] expression,ddl: fix fraction part handle of current_timestamp (#7355) --- ddl/db_test.go | 2 +- executor/executor_test.go | 34 ++++++++++++++++++++++++++++++++++ expression/helper.go | 3 ++- 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 69921cb423071..e134ddadbc615 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1760,7 +1760,7 @@ func (s *testDBSuite) TestTableDDLWithTimeType(c *C) { s.testErrorCode(c, "alter table t change column a aa time(7)", tmysql.ErrTooBigPrecision) s.testErrorCode(c, "alter table t change column a aa datetime(7)", tmysql.ErrTooBigPrecision) s.testErrorCode(c, "alter table t change column a aa timestamp(7)", tmysql.ErrTooBigPrecision) - s.mustExec(c, "alter table t change column a aa timestamp(0)") + s.mustExec(c, "alter table t change column a aa datetime(0)") s.mustExec(c, "drop table t") } diff --git a/executor/executor_test.go b/executor/executor_test.go index e2e53470e550e..ae0156ea61570 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3088,3 +3088,37 @@ func (s *testSuite) TestMaxOneRow(c *C) { err = rs.Close() c.Assert(err, IsNil) } + +func (s *testSuite) TestCurrentTimestampValueSelection(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t,t1") + + tk.MustExec("create table t (id int, t0 timestamp null default current_timestamp, t1 timestamp(1) null default current_timestamp(1), t2 timestamp(2) null default current_timestamp(2) on update current_timestamp(2))") + tk.MustExec("insert into t (id) values (1)") + rs := tk.MustQuery("select t0, t1, t2 from t where id = 1") + t0 := rs.Rows()[0][0].(string) + t1 := rs.Rows()[0][1].(string) + t2 := rs.Rows()[0][2].(string) + c.Assert(len(strings.Split(t0, ".")), Equals, 1) + c.Assert(len(strings.Split(t1, ".")[1]), Equals, 1) + c.Assert(len(strings.Split(t2, ".")[1]), Equals, 2) + tk.MustQuery("select id from t where t0 = ?", t0).Check(testkit.Rows("1")) + tk.MustQuery("select id from t where t1 = ?", t1).Check(testkit.Rows("1")) + tk.MustQuery("select id from t where t2 = ?", t2).Check(testkit.Rows("1")) + time.Sleep(time.Second / 2) + tk.MustExec("update t set t0 = now() where id = 1") + rs = tk.MustQuery("select t2 from t where id = 1") + newT2 := rs.Rows()[0][0].(string) + c.Assert(newT2 != t2, IsTrue) + + tk.MustExec("create table t1 (id int, a timestamp, b timestamp(2), c timestamp(3))") + tk.MustExec("insert into t1 (id, a, b, c) values (1, current_timestamp(2), current_timestamp, current_timestamp(3))") + rs = tk.MustQuery("select a, b, c from t1 where id = 1") + a := rs.Rows()[0][0].(string) + b := rs.Rows()[0][1].(string) + d := rs.Rows()[0][2].(string) + c.Assert(len(strings.Split(a, ".")), Equals, 1) + c.Assert(strings.Split(b, ".")[1], Equals, "00") + c.Assert(len(strings.Split(d, ".")[1]), Equals, 3) +} diff --git a/expression/helper.go b/expression/helper.go index a7bc83fcb7770..813f45e903c85 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -14,6 +14,7 @@ package expression import ( + "math" "strings" "time" @@ -57,7 +58,7 @@ func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int) (d ty case string: upperX := strings.ToUpper(x) if upperX == strings.ToUpper(ast.CurrentTimestamp) { - value.Time = types.FromGoTime(defaultTime) + value.Time = types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-fsp)) * time.Nanosecond)) if tp == mysql.TypeTimestamp { err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().Location()) if err != nil { From 7523566bc9458ce13e7d4730cc2b8169831615d4 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Tue, 14 Aug 2018 13:33:43 +0800 Subject: [PATCH 50/87] executor: CheckIndexRangeExec should set the types of index columns (#7376) --- executor/admin.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/executor/admin.go b/executor/admin.go index 0b1e9af294384..2faf6278eeaee 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -93,10 +93,14 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { col := tCols[ic.Offset] e.cols = append(e.cols, col) } + + colTypeForHandle := e.schema.Columns[len(e.cols)].RetType e.cols = append(e.cols, &model.ColumnInfo{ - ID: model.ExtraHandleID, - Name: model.ExtraHandleName, + ID: model.ExtraHandleID, + Name: model.ExtraHandleName, + FieldType: *colTypeForHandle, }) + e.srcChunk = e.newChunk() dagPB, err := e.buildDAGPB() if err != nil { From d1f258790381f69a7570ab05a2c944844647f327 Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 14 Aug 2018 13:54:16 +0800 Subject: [PATCH 51/87] *: admin check table return tidb specific error code. (#7363) --- executor/admin_test.go | 2 ++ executor/errors.go | 2 ++ executor/executor.go | 4 ++++ mysql/errcode.go | 1 + mysql/errname.go | 7 +++++-- util/admin/admin.go | 17 +++++++++-------- 6 files changed, 23 insertions(+), 10 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 01951190d4c77..f99bafa1336b1 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -17,6 +17,7 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -100,6 +101,7 @@ func (s *testSuite) TestAdminRecoverIndex(c *C) { c.Assert(err, IsNil) _, err = tk.Exec("admin check table admin_test") c.Assert(err, NotNil) + c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) _, err = tk.Exec("admin check index admin_test c2") c.Assert(err, NotNil) diff --git a/executor/errors.go b/executor/errors.go index cfe42da442c65..ec8482ac56a44 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -42,6 +42,7 @@ var ( ErrPasswordFormat = terror.ClassExecutor.New(mysql.ErrPasswordFormat, mysql.MySQLErrName[mysql.ErrPasswordFormat]) ErrCantChangeTxCharacteristics = terror.ClassExecutor.New(mysql.ErrCantChangeTxCharacteristics, mysql.MySQLErrName[mysql.ErrCantChangeTxCharacteristics]) ErrPsManyParam = terror.ClassExecutor.New(mysql.ErrPsManyParam, mysql.MySQLErrName[mysql.ErrPsManyParam]) + ErrAdminCheckTable = terror.ClassExecutor.New(mysql.ErrAdminCheckTable, mysql.MySQLErrName[mysql.ErrAdminCheckTable]) ) func init() { @@ -53,6 +54,7 @@ func init() { mysql.ErrPasswordFormat: mysql.ErrPasswordFormat, mysql.ErrCantChangeTxCharacteristics: mysql.ErrCantChangeTxCharacteristics, mysql.ErrPsManyParam: mysql.ErrPsManyParam, + mysql.ErrAdminCheckTable: mysql.ErrAdminCheckTable, } terror.ErrClassToMySQLCodes[terror.ClassExecutor] = tableMySQLErrCodes } diff --git a/executor/executor.go b/executor/executor.go index 0a5cf7c39d96b..51a126c0d27b6 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -383,6 +383,10 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { } if err != nil { log.Warnf("%v error:%v", t.Name, errors.ErrorStack(err)) + if admin.ErrDataInConsistent.Equal(err) { + return ErrAdminCheckTable.Gen("%v err:%v", t.Name, err) + } + return errors.Errorf("%v err:%v", t.Name, err) } } diff --git a/mysql/errcode.go b/mysql/errcode.go index be74573c1d6bf..dcc24bc5c3799 100644 --- a/mysql/errcode.go +++ b/mysql/errcode.go @@ -894,6 +894,7 @@ const ( // TiDB self-defined errors. ErrMemExceedThreshold = 8001 ErrForUpdateCantRetry = 8002 + ErrAdminCheckTable = 8003 // TiKV/PD errors. ErrPDServerTimeout = 9001 diff --git a/mysql/errname.go b/mysql/errname.go index bc26c7c688f47..5deda27e6716c 100644 --- a/mysql/errname.go +++ b/mysql/errname.go @@ -887,8 +887,11 @@ var MySQLErrName = map[uint16]string{ ErrInvalidJSONPath: "Invalid JSON path expression %s.", ErrInvalidJSONData: "Invalid data type for JSON data", ErrJSONUsedAsKey: "JSON column '%-.192s' cannot be used in key specification.", - ErrMemExceedThreshold: "%s holds %dB memory, exceeds threshold %dB.%s", - ErrForUpdateCantRetry: "[%d] can not retry select for update statement", + + // TiDB errors. + ErrMemExceedThreshold: "%s holds %dB memory, exceeds threshold %dB.%s", + ErrForUpdateCantRetry: "[%d] can not retry select for update statement", + ErrAdminCheckTable: "TiDB admin check table failed.", // TiKV/PD errors. ErrPDServerTimeout: "PD server timeout", diff --git a/util/admin/admin.go b/util/admin/admin.go index a1c6f4885488a..c25cb27d8c2eb 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -366,7 +366,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table vals2 = tables.TruncateIndexValuesIfNeeded(t.Meta(), idx.Meta(), vals2) if kv.ErrNotExist.Equal(err) { record := &RecordData{Handle: h, Values: vals1} - err = errDateNotEqual.Gen("index:%#v != record:%#v", record, nil) + err = ErrDataInConsistent.Gen("index:%#v != record:%#v", record, nil) } if err != nil { return errors.Trace(err) @@ -375,7 +375,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table if !reflect.DeepEqual(vals1, vals2) { record1 := &RecordData{Handle: h, Values: vals1} record2 := &RecordData{Handle: h, Values: vals2} - return errDateNotEqual.Gen("index:%#v != record:%#v", record1, record2) + return ErrDataInConsistent.Gen("index:%#v != record:%#v", record1, record2) } } @@ -410,14 +410,14 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} record2 := &RecordData{Handle: h2, Values: vals1} - return false, errDateNotEqual.Gen("index:%#v != record:%#v", record2, record1) + return false, ErrDataInConsistent.Gen("index:%#v != record:%#v", record2, record1) } if err != nil { return false, errors.Trace(err) } if !isExist { record := &RecordData{Handle: h1, Values: vals1} - return false, errDateNotEqual.Gen("index:%#v != record:%#v", nil, record) + return false, ErrDataInConsistent.Gen("index:%#v != record:%#v", nil, record) } return true, nil @@ -505,7 +505,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. vals2, ok := m[h] if !ok { record := &RecordData{Handle: h, Values: vals} - return false, errDateNotEqual.Gen("data:%#v != record:%#v", nil, record) + return false, ErrDataInConsistent.Gen("data:%#v != record:%#v", nil, record) } if !exact { delete(m, h) @@ -515,7 +515,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. if !reflect.DeepEqual(vals, vals2) { record1 := &RecordData{Handle: h, Values: vals2} record2 := &RecordData{Handle: h, Values: vals} - return false, errDateNotEqual.Gen("data:%#v != record:%#v", record1, record2) + return false, ErrDataInConsistent.Gen("data:%#v != record:%#v", record1, record2) } delete(m, h) @@ -529,7 +529,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. for h, vals := range m { record := &RecordData{Handle: h, Values: vals} - return errDateNotEqual.Gen("data:%#v != record:%#v", record, nil) + return ErrDataInConsistent.Gen("data:%#v != record:%#v", record, nil) } return nil @@ -660,7 +660,8 @@ const ( ) var ( - errDateNotEqual = terror.ClassAdmin.New(codeDataNotEqual, "data isn't equal") + // ErrDataInConsistent indicate that meets inconsistent data. + ErrDataInConsistent = terror.ClassAdmin.New(codeDataNotEqual, "data isn't equal") errRepeatHandle = terror.ClassAdmin.New(codeRepeatHandle, "handle is repeated") errInvalidColumnState = terror.ClassAdmin.New(codeInvalidColumnState, "invalid column state") ) From 360ef5cee790475e9d75cb349e1b47b0a7a4efee Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Tue, 14 Aug 2018 15:22:38 +0800 Subject: [PATCH 52/87] expression: tiny change to extend `SimpleRewriter`'s ability (#7373) --- ddl/partition.go | 4 ++-- expression/builtin_compare_test.go | 2 +- expression/expression.go | 1 + expression/simple_rewriter.go | 37 +++++++++++------------------- table/tables/partition.go | 4 ++-- 5 files changed, 20 insertions(+), 28 deletions(-) diff --git a/ddl/partition.go b/ddl/partition.go index 25f18d95557a9..c9dd67a192658 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -149,7 +149,7 @@ func checkPartitionFuncType(ctx sessionctx.Context, s *ast.CreateTableStmt, cols } } - e, err := expression.ParseSimpleExpr(ctx, buf.String(), tblInfo) + e, err := expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo) if err != nil { return errors.Trace(err) } @@ -297,7 +297,7 @@ func checkRangePartitioningKeysConstraints(ctx sessionctx.Context, s *ast.Create buf := new(bytes.Buffer) s.Partition.Expr.Format(buf) var partkeys []string - e, err := expression.ParseSimpleExpr(ctx, buf.String(), tblInfo) + e, err := expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo) if err != nil { return errors.Trace(err) } diff --git a/expression/builtin_compare_test.go b/expression/builtin_compare_test.go index f2867b9dc4f33..bbeda8c720173 100644 --- a/expression/builtin_compare_test.go +++ b/expression/builtin_compare_test.go @@ -69,7 +69,7 @@ func (s *testEvaluatorSuite) TestCompareFunctionWithRefine(c *C) { } for _, t := range tests { - f, err := ParseSimpleExpr(s.ctx, t.exprStr, tblInfo) + f, err := ParseSimpleExprWithTableInfo(s.ctx, t.exprStr, tblInfo) c.Assert(err, IsNil) c.Assert(f.String(), Equals, t.result) } diff --git a/expression/expression.go b/expression/expression.go index 33bed2842bce8..e4be2df9d798c 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -312,6 +312,7 @@ func ColumnInfos2ColumnsWithDBName(ctx sessionctx.Context, dbName, tblName model TblName: tblName, DBName: dbName, RetType: &col.FieldType, + ID: col.ID, UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), Index: col.Offset, } diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index 7d83049fa15fc..5a5b4450b5c1e 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -27,26 +27,28 @@ import ( type simpleRewriter struct { exprStack - tbl *model.TableInfo - err error - ctx sessionctx.Context + schema *Schema + err error + ctx sessionctx.Context } -// ParseSimpleExpr parses simple expression string to Expression. +// ParseSimpleExprWithTableInfo parses simple expression string to Expression. // The expression string must only reference the column in table Info. -func ParseSimpleExpr(ctx sessionctx.Context, exprStr string, tableInfo *model.TableInfo) (Expression, error) { +func ParseSimpleExprWithTableInfo(ctx sessionctx.Context, exprStr string, tableInfo *model.TableInfo) (Expression, error) { exprStr = "select " + exprStr stmts, err := parser.New().Parse(exprStr, "", "") if err != nil { return nil, errors.Trace(err) } expr := stmts[0].(*ast.SelectStmt).Fields.Fields[0].Expr - return RewriteSimpleExpr(ctx, tableInfo, expr) + return RewriteSimpleExprWithTableInfo(ctx, tableInfo, expr) } -// RewriteSimpleExpr rewrites simple ast.ExprNode to expression.Expression. -func RewriteSimpleExpr(ctx sessionctx.Context, tbl *model.TableInfo, expr ast.ExprNode) (Expression, error) { - rewriter := &simpleRewriter{tbl: tbl, ctx: ctx} +// RewriteSimpleExprWithTableInfo rewrites simple ast.ExprNode to expression.Expression. +func RewriteSimpleExprWithTableInfo(ctx sessionctx.Context, tbl *model.TableInfo, expr ast.ExprNode) (Expression, error) { + dbName := model.NewCIStr(ctx.GetSessionVars().CurrentDB) + columns := ColumnInfos2ColumnsWithDBName(ctx, dbName, tbl.Name, tbl.Columns) + rewriter := &simpleRewriter{ctx: ctx, schema: NewSchema(columns...)} expr.Accept(rewriter) if rewriter.err != nil { return nil, errors.Trace(rewriter.err) @@ -55,20 +57,9 @@ func RewriteSimpleExpr(ctx sessionctx.Context, tbl *model.TableInfo, expr ast.Ex } func (sr *simpleRewriter) rewriteColumn(nodeColName *ast.ColumnNameExpr) (*Column, error) { - tblCols := sr.tbl.Columns - for i, col := range tblCols { - if col.Name.L == nodeColName.Name.Name.L { - return &Column{ - ColName: col.Name, - OrigTblName: sr.tbl.Name, - DBName: model.NewCIStr(sr.ctx.GetSessionVars().CurrentDB), - TblName: sr.tbl.Name, - RetType: &col.FieldType, - ID: col.ID, - UniqueID: sr.ctx.GetSessionVars().AllocPlanColumnID(), - Index: i, - }, nil - } + col := sr.schema.FindColumnByName(nodeColName.Name.Name.L) + if col != nil { + return col, nil } return nil, errBadField.GenByArgs(nodeColName.Name.Name.O, "expression") } diff --git a/table/tables/partition.go b/table/tables/partition.go index 5219f5f42c048..7dce282482653 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -119,7 +119,7 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) { } else { fmt.Fprintf(&buf, "((%s) < (%s))", pi.Expr, pi.Definitions[i].LessThan[0]) } - expr, err := expression.ParseSimpleExpr(ctx, buf.String(), tblInfo) + expr, err := expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo) if err != nil { // If it got an error here, ddl may hang forever, so this error log is important. log.Error("wrong table partition expression:", errors.ErrorStack(err), buf.String()) @@ -131,7 +131,7 @@ func generatePartitionExpr(tblInfo *model.TableInfo) (*PartitionExpr, error) { fmt.Fprintf(&buf, " and ((%s) >= (%s))", pi.Expr, pi.Definitions[i-1].LessThan[0]) } - expr, err = expression.ParseSimpleExpr(ctx, buf.String(), tblInfo) + expr, err = expression.ParseSimpleExprWithTableInfo(ctx, buf.String(), tblInfo) if err != nil { // If it got an error here, ddl may hang forever, so this error log is important. log.Error("wrong table partition expression:", errors.ErrorStack(err), buf.String()) From 1afd12762d14cd809446da6e5a38a0093b72dcc9 Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 14 Aug 2018 17:29:56 +0800 Subject: [PATCH 53/87] plan: fix admin check index panic when index contains pkIsHandle column. (#7350) --- executor/admin_test.go | 8 ++++++++ plan/planbuilder.go | 1 + 2 files changed, 9 insertions(+) diff --git a/executor/admin_test.go b/executor/admin_test.go index f99bafa1336b1..2b09b202966e5 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -472,3 +472,11 @@ func (s *testSuite) TestAdminCheckTable(c *C) { tk.MustExec(`INSERT INTO t VALUES ('keyword','urlprefix','text/ /text');`) tk.MustExec(`admin check table t;`) } + +func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b));") + tk.MustExec("insert into t values(1, 1, 1), (9223372036854775807, 2, 2);") + tk.MustExec("admin check index t idx;") +} diff --git a/plan/planbuilder.go b/plan/planbuilder.go index c202c151f2873..a24fcfe6bedb4 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -409,6 +409,7 @@ func (b *planBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl if idxCol.Name.L == col.Name.L { columns = append(columns, col) schema.Append(&expression.Column{ + ColName: col.Name, UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: &col.FieldType, }) From 34b0c6cb6d1399f13b465f24718970bb4f23bd0a Mon Sep 17 00:00:00 2001 From: liukun <451564319@qq.com> Date: Tue, 14 Aug 2018 19:22:28 +0800 Subject: [PATCH 54/87] mocktikv: add the batch put and batch delete for raw client (#7330) --- store/mockstore/mocktikv/mvcc.go | 2 + store/mockstore/mocktikv/mvcc_leveldb.go | 26 ++++ store/mockstore/mocktikv/rpc.go | 47 ++++++ store/tikv/rawkv.go | 190 ++++++++++++++++++++++- store/tikv/rawkv_test.go | 33 +++- store/tikv/tikvrpc/tikvrpc.go | 30 ++++ 6 files changed, 323 insertions(+), 5 deletions(-) diff --git a/store/mockstore/mocktikv/mvcc.go b/store/mockstore/mocktikv/mvcc.go index 626d1d85475d0..a08045faaf796 100644 --- a/store/mockstore/mocktikv/mvcc.go +++ b/store/mockstore/mocktikv/mvcc.go @@ -437,10 +437,12 @@ type MVCCStore interface { // RawKV is a key-value storage. MVCCStore can be implemented upon it with timestamp encoded into key. type RawKV interface { RawGet(key []byte) []byte + RawBatchGet(keys [][]byte) [][]byte RawScan(startKey, endKey []byte, limit int) []Pair RawPut(key, value []byte) RawBatchPut(keys, values [][]byte) RawDelete(key []byte) + RawBatchDelete(keys [][]byte) RawDeleteRange(startKey, endKey []byte) } diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/mockstore/mocktikv/mvcc_leveldb.go index e773dd144c98e..2f43dcddecf7a 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/mockstore/mocktikv/mvcc_leveldb.go @@ -919,6 +919,20 @@ func (mvcc *MVCCLevelDB) RawGet(key []byte) []byte { return ret } +// RawBatchGet implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawBatchGet(keys [][]byte) [][]byte { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + var values [][]byte + for _, key := range keys { + value, err := mvcc.db.Get(key, nil) + terror.Log(err) + values = append(values, value) + } + return values +} + // RawDelete implements the RawKV interface. func (mvcc *MVCCLevelDB) RawDelete(key []byte) { mvcc.mu.Lock() @@ -927,6 +941,18 @@ func (mvcc *MVCCLevelDB) RawDelete(key []byte) { terror.Log(mvcc.db.Delete(key, nil)) } +// RawBatchDelete implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawBatchDelete(keys [][]byte) { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + batch := &leveldb.Batch{} + for _, key := range keys { + batch.Delete(key) + } + terror.Log(mvcc.db.Write(batch, nil)) +} + // RawScan implements the RawKV interface. func (mvcc *MVCCLevelDB) RawScan(startKey, endKey []byte, limit int) []Pair { mvcc.mu.Lock() diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 1bbafa40e535c..0877ca0cddd63 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -377,6 +377,29 @@ func (h *rpcHandler) handleKvRawGet(req *kvrpcpb.RawGetRequest) *kvrpcpb.RawGetR } } +func (h *rpcHandler) handleKvRawBatchGet(req *kvrpcpb.RawBatchGetRequest) *kvrpcpb.RawBatchGetResponse { + rawKV, ok := h.mvccStore.(RawKV) + if !ok { + // TODO should we add error ? + return &kvrpcpb.RawBatchGetResponse{ + RegionError: &errorpb.Error{ + Message: "not implemented", + }, + } + } + values := rawKV.RawBatchGet(req.Keys) + kvPairs := make([]*kvrpcpb.KvPair, len(values)) + for i, key := range req.Keys { + kvPairs[i] = &kvrpcpb.KvPair{ + Key: key, + Value: values[i], + } + } + return &kvrpcpb.RawBatchGetResponse{ + Pairs: kvPairs, + } +} + func (h *rpcHandler) handleKvRawPut(req *kvrpcpb.RawPutRequest) *kvrpcpb.RawPutResponse { rawKV, ok := h.mvccStore.(RawKV) if !ok { @@ -416,6 +439,17 @@ func (h *rpcHandler) handleKvRawDelete(req *kvrpcpb.RawDeleteRequest) *kvrpcpb.R return &kvrpcpb.RawDeleteResponse{} } +func (h *rpcHandler) handleKvRawBatchDelete(req *kvrpcpb.RawBatchDeleteRequest) *kvrpcpb.RawBatchDeleteResponse { + rawKV, ok := h.mvccStore.(RawKV) + if !ok { + return &kvrpcpb.RawBatchDeleteResponse{ + Error: "not implemented", + } + } + rawKV.RawBatchDelete(req.Keys) + return &kvrpcpb.RawBatchDeleteResponse{} +} + func (h *rpcHandler) handleKvRawDeleteRange(req *kvrpcpb.RawDeleteRangeRequest) *kvrpcpb.RawDeleteRangeResponse { rawKV, ok := h.mvccStore.(RawKV) if !ok { @@ -625,6 +659,13 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R return resp, nil } resp.RawGet = handler.handleKvRawGet(r) + case tikvrpc.CmdRawBatchGet: + r := req.RawBatchGet + if err := handler.checkRequest(reqCtx, r.Size()); err != nil { + resp.RawBatchGet = &kvrpcpb.RawBatchGetResponse{RegionError: err} + return resp, nil + } + resp.RawBatchGet = handler.handleKvRawBatchGet(r) case tikvrpc.CmdRawPut: r := req.RawPut if err := handler.checkRequest(reqCtx, r.Size()); err != nil { @@ -646,6 +687,12 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R return resp, nil } resp.RawDelete = handler.handleKvRawDelete(r) + case tikvrpc.CmdRawBatchDelete: + r := req.RawBatchDelete + if err := handler.checkRequest(reqCtx, r.Size()); err != nil { + resp.RawBatchDelete = &kvrpcpb.RawBatchDeleteResponse{RegionError: err} + } + resp.RawBatchDelete = handler.handleKvRawBatchDelete(r) case tikvrpc.CmdRawDeleteRange: r := req.RawDeleteRange if err := handler.checkRequest(reqCtx, r.Size()); err != nil { diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 322bbf4518af3..41b72a1986859 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -35,7 +35,12 @@ var ( ErrMaxScanLimitExceeded = errors.New("limit should be less than MaxRawKVScanLimit") ) -const rawBatchPutSize = 16 * 1024 +const ( + // rawBatchPutSize is the maximum size limit for rawkv each batch put request. + rawBatchPutSize = 16 * 1024 + // rawBatchPairCount is the maximum limit for rawkv each batch get/delete request. + rawBatchPairCount = 512 +) // RawKVClient is a client of TiKV server which is used as a key-value storage, // only GET/PUT/DELETE commands are supported. @@ -103,6 +108,36 @@ func (c *RawKVClient) Get(key []byte) ([]byte, error) { return cmdResp.Value, nil } +// BatchGet queries values with the keys. +func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { + start := time.Now() + defer func() { + metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_get").Observe(time.Since(start).Seconds()) + }() + + bo := NewBackoffer(context.Background(), rawkvMaxBackoff) + resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchGet) + if err != nil { + return nil, errors.Trace(err) + } + + cmdResp := resp.RawBatchGet + if cmdResp == nil { + return nil, errors.Trace(ErrBodyMissing) + } + + keyToValue := make(map[string][]byte, len(keys)) + for _, pair := range cmdResp.Pairs { + keyToValue[string(pair.Key)] = pair.Value + } + + values := make([][]byte, len(keys)) + for i, key := range keys { + values[i] = keyToValue[string(key)] + } + return values, nil +} + // Put stores a key-value pair to TiKV. func (c *RawKVClient) Put(key, value []byte) error { start := time.Now() @@ -180,6 +215,28 @@ func (c *RawKVClient) Delete(key []byte) error { return nil } +// BatchDelete deletes key-value pairs from TiKV +func (c *RawKVClient) BatchDelete(keys [][]byte) error { + start := time.Now() + defer func() { + metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_delete").Observe(time.Since(start).Seconds()) + }() + + bo := NewBackoffer(context.Background(), rawkvMaxBackoff) + resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchDelete) + if err != nil { + return errors.Trace(err) + } + cmdResp := resp.RawBatchDelete + if cmdResp == nil { + return errors.Trace(ErrBodyMissing) + } + if cmdResp.GetError() != "" { + return errors.New(cmdResp.GetError()) + } + return nil +} + // DeleteRange deletes all key-value pairs in a range from TiKV func (c *RawKVClient) DeleteRange(startKey []byte, endKey []byte) error { start := time.Now() @@ -278,6 +335,115 @@ func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request) (*tikvrpc.Respon } } +func (c *RawKVClient) sendBatchReq(bo *Backoffer, keys [][]byte, cmdType tikvrpc.CmdType) (*tikvrpc.Response, error) { // split the keys + groups, _, err := c.regionCache.GroupKeysByRegion(bo, keys) + if err != nil { + return nil, errors.Trace(err) + } + + var batches []batch + for regionID, groupKeys := range groups { + batches = appendKeyBatches(batches, regionID, groupKeys, rawBatchPairCount) + } + bo, cancel := bo.Fork() + ches := make(chan singleBatchResp, len(batches)) + for _, batch := range batches { + batch1 := batch + rawKVClientGP.Go(func() { + singleBatchBackoffer, singleBatchCancel := bo.Fork() + defer singleBatchCancel() + ches <- c.doBatchReq(singleBatchBackoffer, batch1, cmdType) + }) + } + + var firstError error + var resp *tikvrpc.Response + switch cmdType { + case tikvrpc.CmdRawBatchGet: + resp = &tikvrpc.Response{Type: tikvrpc.CmdRawBatchGet, RawBatchGet: &kvrpcpb.RawBatchGetResponse{}} + case tikvrpc.CmdRawBatchDelete: + resp = &tikvrpc.Response{Type: tikvrpc.CmdRawBatchDelete, RawBatchDelete: &kvrpcpb.RawBatchDeleteResponse{}} + } + for i := 0; i < len(batches); i++ { + singleResp, ok := <-ches + if ok { + if singleResp.err != nil { + cancel() + if firstError == nil { + firstError = singleResp.err + } + } else if cmdType == tikvrpc.CmdRawBatchGet { + cmdResp := singleResp.resp.RawBatchGet + resp.RawBatchGet.Pairs = append(resp.RawBatchGet.Pairs, cmdResp.Pairs...) + } + } + } + + return resp, firstError +} + +func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.CmdType) singleBatchResp { + var req *tikvrpc.Request + switch cmdType { + case tikvrpc.CmdRawBatchGet: + req = &tikvrpc.Request{ + Type: cmdType, + RawBatchGet: &kvrpcpb.RawBatchGetRequest{ + Keys: batch.keys, + }, + } + case tikvrpc.CmdRawBatchDelete: + req = &tikvrpc.Request{ + Type: cmdType, + RawBatchDelete: &kvrpcpb.RawBatchDeleteRequest{ + Keys: batch.keys, + }, + } + } + + sender := NewRegionRequestSender(c.regionCache, c.rpcClient) + resp, err := sender.SendReq(bo, req, batch.regionID, readTimeoutShort) + + batchResp := singleBatchResp{} + if err != nil { + batchResp.err = errors.Trace(err) + return batchResp + } + regionErr, err := resp.GetRegionError() + if err != nil { + batchResp.err = errors.Trace(err) + return batchResp + } + if regionErr != nil { + err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + batchResp.err = errors.Trace(err) + return batchResp + } + resp, err = c.sendBatchReq(bo, batch.keys, cmdType) + batchResp.resp = resp + batchResp.err = err + return batchResp + } + + switch cmdType { + case tikvrpc.CmdRawBatchGet: + batchResp.resp = resp + case tikvrpc.CmdRawBatchDelete: + cmdResp := resp.RawBatchDelete + if cmdResp == nil { + batchResp.err = errors.Trace(ErrBodyMissing) + return batchResp + } + if cmdResp.GetError() != "" { + batchResp.err = errors.New(cmdResp.GetError()) + return batchResp + } + batchResp.resp = resp + } + return batchResp +} + // sendDeleteRangeReq sends a raw delete range request and returns the response and the actual endKey. // If the given range spans over more than one regions, the actual endKey is the end of the first region. // We can't use sendReq directly, because we need to know the end of the region before we send the request @@ -360,6 +526,23 @@ func (c *RawKVClient) sendBatchPut(bo *Backoffer, keys, values [][]byte) error { return errors.Trace(err) } +func appendKeyBatches(batches []batch, regionID RegionVerID, groupKeys [][]byte, limit int) []batch { + var keys [][]byte + for start, count := 0, 0; start < len(groupKeys); start++ { + if count > limit { + batches = append(batches, batch{regionID: regionID, keys: keys}) + keys = make([][]byte, 0, limit) + count = 0 + } + keys = append(keys, groupKeys[start]) + count++ + } + if len(keys) != 0 { + batches = append(batches, batch{regionID: regionID, keys: keys}) + } + return batches +} + func appendBatches(batches []batch, regionID RegionVerID, groupKeys [][]byte, keyToValue map[string][]byte, limit int) []batch { var start, size int var keys, values [][]byte @@ -429,3 +612,8 @@ type batch struct { keys [][]byte values [][]byte } + +type singleBatchResp struct { + resp *tikvrpc.Response + err error +} diff --git a/store/tikv/rawkv_test.go b/store/tikv/rawkv_test.go index 638a396af07a7..9111af86e24ec 100644 --- a/store/tikv/rawkv_test.go +++ b/store/tikv/rawkv_test.go @@ -55,6 +55,16 @@ func (s *testRawKVSuite) mustNotExist(c *C, key []byte) { c.Assert(v, IsNil) } +func (s *testRawKVSuite) mustBatchNotExist(c *C, keys [][]byte) { + values, err := s.client.BatchGet(keys) + c.Assert(err, IsNil) + c.Assert(values, NotNil) + c.Assert(len(keys), Equals, len(values)) + for _, value := range values { + c.Assert([]byte{}, BytesEquals, value) + } +} + func (s *testRawKVSuite) mustGet(c *C, key, value []byte) { v, err := s.client.Get(key) c.Assert(err, IsNil) @@ -62,6 +72,16 @@ func (s *testRawKVSuite) mustGet(c *C, key, value []byte) { c.Assert(v, BytesEquals, value) } +func (s *testRawKVSuite) mustBatchGet(c *C, keys, values [][]byte) { + checkValues, err := s.client.BatchGet(keys) + c.Assert(err, IsNil) + c.Assert(checkValues, NotNil) + c.Assert(len(keys), Equals, len(checkValues)) + for i := range keys { + c.Check(values[i], BytesEquals, checkValues[i]) + } +} + func (s *testRawKVSuite) mustPut(c *C, key, value []byte) { err := s.client.Put(key, value) c.Assert(err, IsNil) @@ -77,6 +97,11 @@ func (s *testRawKVSuite) mustDelete(c *C, key []byte) { c.Assert(err, IsNil) } +func (s *testRawKVSuite) mustBatchDelete(c *C, keys [][]byte) { + err := s.client.BatchDelete(keys) + c.Assert(err, IsNil) +} + func (s *testRawKVSuite) mustScan(c *C, startKey string, limit int, expect ...string) { keys, values, err := s.client.Scan([]byte(startKey), limit) c.Assert(err, IsNil) @@ -132,7 +157,7 @@ func (s *testRawKVSuite) TestSimple(c *C) { c.Assert(err, NotNil) } -func (s *testRawKVSuite) TestBatchPut(c *C) { +func (s *testRawKVSuite) TestRawBatch(c *C) { testNum := 0 size := 0 var testKeys [][]byte @@ -150,9 +175,9 @@ func (s *testRawKVSuite) TestBatchPut(c *C) { err := s.split(c, "", fmt.Sprint("key", testNum/2)) c.Assert(err, IsNil) s.mustBatchPut(c, testKeys, testValues) - for i := 0; i < testNum; i++ { - s.mustGet(c, testKeys[i], testValues[i]) - } + s.mustBatchGet(c, testKeys, testValues) + s.mustBatchDelete(c, testKeys) + s.mustBatchNotExist(c, testKeys) } func (s *testRawKVSuite) TestSplit(c *C) { diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index 3e12dfd58c3a9..e9c7eb2189d0c 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -45,9 +45,11 @@ const ( CmdDeleteRange CmdRawGet CmdType = 256 + iota + CmdRawBatchGet CmdRawPut CmdRawBatchPut CmdRawDelete + CmdRawBatchDelete CmdRawDeleteRange CmdRawScan @@ -85,12 +87,16 @@ func (t CmdType) String() string { return "DeleteRange" case CmdRawGet: return "RawGet" + case CmdRawBatchGet: + return "RawBatchGet" case CmdRawPut: return "RawPut" case CmdRawBatchPut: return "RawBatchPut" case CmdRawDelete: return "RawDelete" + case CmdRawBatchDelete: + return "RawBatchDelete" case CmdRawDeleteRange: return "RawDeleteRange" case CmdRawScan: @@ -125,9 +131,11 @@ type Request struct { GC *kvrpcpb.GCRequest DeleteRange *kvrpcpb.DeleteRangeRequest RawGet *kvrpcpb.RawGetRequest + RawBatchGet *kvrpcpb.RawBatchGetRequest RawPut *kvrpcpb.RawPutRequest RawBatchPut *kvrpcpb.RawBatchPutRequest RawDelete *kvrpcpb.RawDeleteRequest + RawBatchDelete *kvrpcpb.RawBatchDeleteRequest RawDeleteRange *kvrpcpb.RawDeleteRangeRequest RawScan *kvrpcpb.RawScanRequest Cop *coprocessor.Request @@ -151,9 +159,11 @@ type Response struct { GC *kvrpcpb.GCResponse DeleteRange *kvrpcpb.DeleteRangeResponse RawGet *kvrpcpb.RawGetResponse + RawBatchGet *kvrpcpb.RawBatchGetResponse RawPut *kvrpcpb.RawPutResponse RawBatchPut *kvrpcpb.RawBatchPutResponse RawDelete *kvrpcpb.RawDeleteResponse + RawBatchDelete *kvrpcpb.RawBatchDeleteResponse RawDeleteRange *kvrpcpb.RawDeleteRangeResponse RawScan *kvrpcpb.RawScanResponse Cop *coprocessor.Response @@ -205,12 +215,16 @@ func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error { req.DeleteRange.Context = ctx case CmdRawGet: req.RawGet.Context = ctx + case CmdRawBatchGet: + req.RawBatchGet.Context = ctx case CmdRawPut: req.RawPut.Context = ctx case CmdRawBatchPut: req.RawBatchPut.Context = ctx case CmdRawDelete: req.RawDelete.Context = ctx + case CmdRawBatchDelete: + req.RawBatchDelete.Context = ctx case CmdRawDeleteRange: req.RawDeleteRange.Context = ctx case CmdRawScan: @@ -285,6 +299,10 @@ func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) { resp.RawGet = &kvrpcpb.RawGetResponse{ RegionError: e, } + case CmdRawBatchGet: + resp.RawBatchGet = &kvrpcpb.RawBatchGetResponse{ + RegionError: e, + } case CmdRawPut: resp.RawPut = &kvrpcpb.RawPutResponse{ RegionError: e, @@ -297,6 +315,10 @@ func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) { resp.RawDelete = &kvrpcpb.RawDeleteResponse{ RegionError: e, } + case CmdRawBatchDelete: + resp.RawBatchDelete = &kvrpcpb.RawBatchDeleteResponse{ + RegionError: e, + } case CmdRawDeleteRange: resp.RawDeleteRange = &kvrpcpb.RawDeleteRangeResponse{ RegionError: e, @@ -361,12 +383,16 @@ func (resp *Response) GetRegionError() (*errorpb.Error, error) { e = resp.DeleteRange.GetRegionError() case CmdRawGet: e = resp.RawGet.GetRegionError() + case CmdRawBatchGet: + e = resp.RawBatchGet.GetRegionError() case CmdRawPut: e = resp.RawPut.GetRegionError() case CmdRawBatchPut: e = resp.RawBatchPut.GetRegionError() case CmdRawDelete: e = resp.RawDelete.GetRegionError() + case CmdRawBatchDelete: + e = resp.RawBatchDelete.GetRegionError() case CmdRawDeleteRange: e = resp.RawDeleteRange.GetRegionError() case CmdRawScan: @@ -419,12 +445,16 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp resp.DeleteRange, err = client.KvDeleteRange(ctx, req.DeleteRange) case CmdRawGet: resp.RawGet, err = client.RawGet(ctx, req.RawGet) + case CmdRawBatchGet: + resp.RawBatchGet, err = client.RawBatchGet(ctx, req.RawBatchGet) case CmdRawPut: resp.RawPut, err = client.RawPut(ctx, req.RawPut) case CmdRawBatchPut: resp.RawBatchPut, err = client.RawBatchPut(ctx, req.RawBatchPut) case CmdRawDelete: resp.RawDelete, err = client.RawDelete(ctx, req.RawDelete) + case CmdRawBatchDelete: + resp.RawBatchDelete, err = client.RawBatchDelete(ctx, req.RawBatchDelete) case CmdRawDeleteRange: resp.RawDeleteRange, err = client.RawDeleteRange(ctx, req.RawDeleteRange) case CmdRawScan: From 5404e2eaae27b9ce31264a4eff0a41aa92ffe328 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 14 Aug 2018 21:44:57 +0800 Subject: [PATCH 55/87] ddl: add admin check before drop table using building flag. (#7343) --- Makefile | 15 ++++++++++++++- config/config.go | 10 ++++++++++ executor/admin_test.go | 5 +++++ executor/ddl.go | 13 +++++++++++++ executor/executor.go | 2 +- executor/executor_test.go | 12 ++++++++++-- util/printer/printer.go | 1 + 7 files changed, 54 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index 9f47f35540f38..64c93de20d7b3 100644 --- a/Makefile +++ b/Makefile @@ -34,12 +34,18 @@ LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitHash=$(shell git rev- LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" LDFLAGS += -X "github.com/pingcap/tidb/util/printer.GoVersion=$(shell go version)" +TEST_LDFLAGS = -X "github.com/pingcap/tidb/config.checkBeforeDropLDFlag=1" + +CHECK_LDFLAGS += $(LDFLAGS) ${TEST_LDFLAGS} + TARGET = "" .PHONY: all build update parser clean todo test gotest interpreter server dev benchkv benchraw check parserlib checklist default: server buildsucc +server-admin-check: server_check buildsucc + buildsucc: @echo Build TiDB Server successfully! @@ -141,7 +147,7 @@ ifeq ("$(TRAVIS_COVERAGE)", "1") else @echo "Running in native mode." @export log_level=error; \ - $(GOTEST) -cover $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } + $(GOTEST) -ldflags '$(TEST_LDFLAGS)' -cover $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; } endif @$(GOFAIL_DISABLE) @@ -178,6 +184,13 @@ else $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -o '$(TARGET)' tidb-server/main.go endif +server_check: parserlib +ifeq ($(TARGET), "") + $(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o bin/tidb-server tidb-server/main.go +else + $(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o '$(TARGET)' tidb-server/main.go +endif + benchkv: $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/benchkv cmd/benchkv/main.go diff --git a/config/config.go b/config/config.go index 036ca3977700a..7f2ce64441a9d 100644 --- a/config/config.go +++ b/config/config.go @@ -36,6 +36,10 @@ var ( "mocktikv": true, "tikv": true, } + // checkTableBeforeDrop enable to execute `admin check table` before `drop table`. + CheckTableBeforeDrop = false + // checkBeforeDropLDFlag is a go build flag. + checkBeforeDropLDFlag = "None" ) // Config contains configuration options. @@ -374,6 +378,12 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { return ret } +func init() { + if checkBeforeDropLDFlag == "1" { + CheckTableBeforeDrop = true + } +} + // The following constants represents the valid action configurations for OOMAction. // NOTE: Althrough the values is case insensitiv, we should use lower-case // strings because the configuration value will be transformed to lower-case diff --git a/executor/admin_test.go b/executor/admin_test.go index 2b09b202966e5..87f62784a3676 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -471,6 +471,11 @@ func (s *testSuite) TestAdminCheckTable(c *C) { INDEX indexIDname (ID(8),name(8)));`) tk.MustExec(`INSERT INTO t VALUES ('keyword','urlprefix','text/ /text');`) tk.MustExec(`admin check table t;`) + + tk.MustExec("use mysql") + tk.MustExec(`admin check table test.t;`) + _, err := tk.Exec("admin check table t") + c.Assert(err, NotNil) } func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) { diff --git a/executor/ddl.go b/executor/ddl.go index ab0a1055f496d..ced59b52fb302 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -14,10 +14,12 @@ package executor import ( + "fmt" "strings" "github.com/juju/errors" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/model" @@ -25,6 +27,8 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -194,6 +198,15 @@ func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { return errors.Trace(err) } + if config.CheckTableBeforeDrop { + log.Warnf("admin check table `%s`.`%s` before drop.", fullti.Schema.O, fullti.Name.O) + sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O) + _, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) + if err != nil { + return errors.Trace(err) + } + } + err = domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, fullti) if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { notExistTables = append(notExistTables, fullti.String()) diff --git a/executor/executor.go b/executor/executor.go index 51a126c0d27b6..552e7a3a7ba7d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -369,8 +369,8 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } defer func() { e.done = true }() - dbName := model.NewCIStr(e.ctx.GetSessionVars().CurrentDB) for _, t := range e.tables { + dbName := t.DBInfo.Name tb, err := e.is.TableByName(dbName, t.Name) if err != nil { return errors.Trace(err) diff --git a/executor/executor_test.go b/executor/executor_test.go index ae0156ea61570..cc97198beebef 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -233,9 +233,17 @@ func (s *testSuite) TestAdmin(c *C) { c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) - r, err = tk.Exec("admin check table admin_test") - c.Assert(err, NotNil) + r, err_admin := tk.Exec("admin check table admin_test") + c.Assert(err_admin, NotNil) + + if config.CheckTableBeforeDrop { + r, err = tk.Exec("drop table admin_test") + c.Assert(err.Error(), Equals, err_admin.Error()) + // Drop inconsistency index. + tk.MustExec("alter table admin_test drop index c1") + tk.MustExec("admin check table admin_test") + } // checksum table test tk.MustExec("create table checksum_with_index (id int, count int, PRIMARY KEY(id), KEY(count))") tk.MustExec("create table checksum_without_index (id int, count int, PRIMARY KEY(id))") diff --git a/util/printer/printer.go b/util/printer/printer.go index 28b01998843a6..b3f57cc020d8a 100644 --- a/util/printer/printer.go +++ b/util/printer/printer.go @@ -43,6 +43,7 @@ func PrintTiDBInfo() { log.Infof("UTC Build Time: %s", TiDBBuildTS) log.Infof("GoVersion: %s", GoVersion) log.Infof("Race Enabled: %v", israce.RaceEnabled) + log.Infof("Check Table Before Drop: %v", config.CheckTableBeforeDrop) log.Infof("TiKV Min Version: %s", TiKVMinVersion) configJSON, err := json.Marshal(config.GetGlobalConfig()) if err != nil { From f18176f806db50bcfbf39be4dfddad59b1ff8f84 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 15 Aug 2018 09:24:58 +0800 Subject: [PATCH 56/87] util/testkit: make the msg of failed check more readable (#7386) --- executor/simple_test.go | 3 ++- util/testkit/testkit.go | 13 ++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/executor/simple_test.go b/executor/simple_test.go index 2485f3c02f903..bc2b9d505d091 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -179,7 +179,8 @@ func (s *testSuite) TestUser(c *C) { tk.MustExec(dropUserSQL) tk.MustQuery("select * from mysql.db").Check(testkit.Rows( "localhost test testDB Y Y Y Y Y Y Y N Y Y N N N N N N Y N N", - "localhost test testDB1 Y Y Y Y Y Y Y N Y Y N N N N N N Y N N] [% dddb_% dduser Y Y Y Y Y Y Y N Y Y N N N N N N Y N N", + "localhost test testDB1 Y Y Y Y Y Y Y N Y Y N N N N N N Y N N", + "% dddb_% dduser Y Y Y Y Y Y Y N Y Y N N N N N N Y N N", "% test test Y N N N N N N N N N N N N N N N N N N", "localhost test testDBRevoke N N N N N N N N N N N N N N N N N N N", )) diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 29523b57b0666..4da9712f20860 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -14,6 +14,7 @@ package testkit import ( + "bytes" "fmt" "sort" "sync/atomic" @@ -43,9 +44,15 @@ type Result struct { // Check asserts the result equals the expected results. func (res *Result) Check(expected [][]interface{}) { - got := fmt.Sprintf("%s", res.rows) - need := fmt.Sprintf("%s", expected) - res.c.Assert(got, check.Equals, need, res.comment) + resBuff := bytes.NewBufferString("") + for _, row := range res.rows { + fmt.Fprintf(resBuff, "%s\n", row) + } + needBuff := bytes.NewBufferString("") + for _, row := range expected { + fmt.Fprintf(needBuff, "%s\n", row) + } + res.c.Assert(resBuff.String(), check.Equals, needBuff.String(), res.comment) } // CheckAt asserts the result of selected columns equals the expected results. From 016006f5c5eac498db879d4643187d6ff550198c Mon Sep 17 00:00:00 2001 From: mengnan <40080929+supernan1994@users.noreply.github.com> Date: Wed, 15 Aug 2018 10:49:20 +0800 Subject: [PATCH 57/87] expression: handle max_allowed_packet warnings for to_base64 function. (#7266) --- expression/builtin_string.go | 30 +++++++++++--- expression/builtin_string_test.go | 69 +++++++++++++++++++++++++++++++ 2 files changed, 93 insertions(+), 6 deletions(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 4998aa24885c8..100e78a2ac6d7 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -232,7 +232,7 @@ func (b *builtinASCIISig) Clone() builtinFunc { return newSig } -// eval evals a builtinASCIISig. +// evalInt evals a builtinASCIISig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_ascii func (b *builtinASCIISig) evalInt(row chunk.Row) (int64, bool, error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) @@ -285,6 +285,7 @@ func (b *builtinConcatSig) Clone() builtinFunc { return newSig } +// evalString evals a builtinConcatSig // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_concat func (b *builtinConcatSig) evalString(row chunk.Row) (d string, isNull bool, err error) { var s []byte @@ -568,7 +569,7 @@ func (b *builtinRepeatSig) Clone() builtinFunc { return newSig } -// eval evals a builtinRepeatSig. +// evalString evals a builtinRepeatSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_repeat func (b *builtinRepeatSig) evalString(row chunk.Row) (d string, isNull bool, err error) { str, isNull, err := b.args[0].EvalString(b.ctx, row) @@ -1515,6 +1516,7 @@ type trimFunctionClass struct { baseFunctionClass } +// getFunction sets trim built-in function signature. // The syntax of trim in mysql is 'TRIM([{BOTH | LEADING | TRAILING} [remstr] FROM] str), TRIM([remstr FROM] str)', // but we wil convert it into trim(str), trim(str, remstr) and trim(str, remstr, direction) in AST. func (c *trimFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { @@ -2482,8 +2484,8 @@ func (b *builtinOctStringSig) Clone() builtinFunc { return newSig } -// // evalString evals OCT(N). -// // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_oct +// evalString evals OCT(N). +// See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_oct func (b *builtinOctStringSig) evalString(row chunk.Row) (string, bool, error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -2999,17 +3001,26 @@ func (c *toBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Expre } bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) bf.tp.Flen = base64NeededEncodedLength(bf.args[0].GetType().Flen) - sig := &builtinToBase64Sig{bf} + + valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) + maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + + sig := &builtinToBase64Sig{bf, maxAllowedPacket} return sig, nil } type builtinToBase64Sig struct { baseBuiltinFunc + maxAllowedPacket uint64 } func (b *builtinToBase64Sig) Clone() builtinFunc { newSig := &builtinToBase64Sig{} newSig.cloneFrom(&b.baseBuiltinFunc) + newSig.maxAllowedPacket = b.maxAllowedPacket return newSig } @@ -3043,7 +3054,14 @@ func (b *builtinToBase64Sig) evalString(row chunk.Row) (d string, isNull bool, e if isNull || err != nil { return "", isNull, errors.Trace(err) } - + needEncodeLen := base64NeededEncodedLength(len(str)) + if needEncodeLen == -1 { + return "", true, nil + } + if needEncodeLen > int(b.maxAllowedPacket) { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnAllowedPacketOverflowed.GenByArgs("to_base64", b.maxAllowedPacket)) + return "", true, nil + } if b.tp.Flen == -1 || b.tp.Flen > mysql.MaxBlobWidth { return "", true, nil } diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 3d067da53a335..7341f7f1055dc 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -23,6 +23,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/charset" @@ -1879,6 +1880,74 @@ func (s *testEvaluatorSuite) TestToBase64(c *C) { c.Assert(err, IsNil) } +func (s *testEvaluatorSuite) TestToBase64Sig(c *C) { + colTypes := []*types.FieldType{ + {Tp: mysql.TypeVarchar}, + } + + tests := []struct { + args string + expect string + isNil bool + maxAllowPacket uint64 + }{ + {"abc", "YWJj", false, 4}, + {"abc", "", true, 3}, + { + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", + "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0\nNTY3ODkrLw==", + false, + 89, + }, + { + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", + "", + true, + 88, + }, + { + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", + "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0\nNTY3ODkrL0FCQ0RFRkdISUpLTE1OT1BRUlNUVVZXWFlaYWJjZGVmZ2hpamtsbW5vcHFyc3R1dnd4\neXowMTIzNDU2Nzg5Ky9BQkNERUZHSElKS0xNTk9QUVJTVFVWV1hZWmFiY2RlZmdoaWprbG1ub3Bx\ncnN0dXZ3eHl6MDEyMzQ1Njc4OSsv", + false, + 259, + }, + { + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", + "", + true, + 258, + }, + } + + args := []Expression{ + &Column{Index: 0, RetType: colTypes[0]}, + } + + for _, test := range tests { + resultType := &types.FieldType{Tp: mysql.TypeVarchar, Flen: base64NeededEncodedLength(len(test.args))} + base := baseBuiltinFunc{args: args, ctx: s.ctx, tp: resultType} + toBase64 := &builtinToBase64Sig{base, test.maxAllowPacket} + + input := chunk.NewChunkWithCapacity(colTypes, 1) + input.AppendString(0, test.args) + res, isNull, err := toBase64.evalString(input.GetRow(0)) + c.Assert(err, IsNil) + if test.isNil { + c.Assert(isNull, IsTrue) + + warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, 1) + lastWarn := warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue) + s.ctx.GetSessionVars().StmtCtx.SetWarnings([]stmtctx.SQLWarn{}) + + } else { + c.Assert(isNull, IsFalse) + } + c.Assert(res, Equals, test.expect) + } +} + func (s *testEvaluatorSuite) TestStringRight(c *C) { defer testleak.AfterTest(c)() fc := funcs[ast.Right] From 4684eec52182a1f981003859baf4cab83745896d Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Wed, 15 Aug 2018 11:05:43 +0800 Subject: [PATCH 58/87] parser: support to character option to load data statement (#7391) --- parser/parser.y | 16 ++++++++++------ parser/parser_test.go | 4 ++++ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/parser/parser.y b/parser/parser.y index 80b283c5a9a53..73bc66d76c556 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -6821,25 +6821,29 @@ RevokeStmt: * See https://dev.mysql.com/doc/refman/5.7/en/load-data.html *******************************************************************************************/ LoadDataStmt: - "LOAD" "DATA" LocalOpt "INFILE" stringLit "INTO" "TABLE" TableName Fields Lines ColumnNameListOptWithBrackets + "LOAD" "DATA" LocalOpt "INFILE" stringLit "INTO" "TABLE" TableName CharsetOpt Fields Lines ColumnNameListOptWithBrackets { x := &ast.LoadDataStmt{ Path: $5, Table: $8.(*ast.TableName), - Columns: $11.([]*ast.ColumnName), + Columns: $12.([]*ast.ColumnName), } if $3 != nil { x.IsLocal = true } - if $9 != nil { - x.FieldsInfo = $9.(*ast.FieldsClause) - } if $10 != nil { - x.LinesInfo = $10.(*ast.LinesClause) + x.FieldsInfo = $10.(*ast.FieldsClause) + } + if $11 != nil { + x.LinesInfo = $11.(*ast.LinesClause) } $$ = x } +CharsetOpt: + {} +| "CHARACTER" "SET" CharsetName + LocalOpt: { $$ = nil diff --git a/parser/parser_test.go b/parser/parser_test.go index 6ec18688bf07d..b38e1e2de34b0 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -358,6 +358,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { // load data {"load data infile '/tmp/t.csv' into table t", true}, + {"load data infile '/tmp/t.csv' into table t character set utf8", true}, {"load data infile '/tmp/t.csv' into table t fields terminated by 'ab'", true}, {"load data infile '/tmp/t.csv' into table t columns terminated by 'ab'", true}, {"load data infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b'", true}, @@ -371,6 +372,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"load data local infile '/tmp/t.csv' into table t columns terminated by 'ab'", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b'", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' escaped by '*'", true}, + {"load data local infile '/tmp/t.csv' into table t character set utf8 fields terminated by 'ab' enclosed by 'b' escaped by '*'", true}, {"load data local infile '/tmp/t.csv' into table t lines starting by 'ab'", true}, {"load data local infile '/tmp/t.csv' into table t lines starting by 'ab' terminated by 'xy'", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' lines terminated by 'xy'", true}, @@ -381,8 +383,10 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"load data local infile '/tmp/t.csv' into table t columns terminated by 'ab' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' enclosed by 'b' escaped by '*' (a,b)", true}, + {"load data local infile '/tmp/t.csv' into table t character set utf8 fields terminated by 'ab' enclosed by 'b' escaped by '*' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t lines starting by 'ab' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t lines starting by 'ab' terminated by 'xy' (a,b)", true}, + {"load data local infile '/tmp/t.csv' into table t character set utf8 fields terminated by 'ab' lines terminated by 'xy' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t fields terminated by 'ab' lines terminated by 'xy' (a,b)", true}, {"load data local infile '/tmp/t.csv' into table t (a,b) fields terminated by 'ab'", false}, From 9fc67b9a9f5094532a785853cd0249185947ac7c Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 15 Aug 2018 12:51:00 +0800 Subject: [PATCH 59/87] store TiDB server info to PD and add http api handle (#7082) --- ddl/ddl.go | 7 ++ ddl/syncer.go | 36 +++--- docs/tidb_http_api.md | 12 ++ domain/domain.go | 16 +++ domain/info.go | 220 ++++++++++++++++++++++++++++++++++++ server/http_handler.go | 83 ++++++++++++++ server/http_handler_test.go | 62 ++++++++++ server/http_status.go | 6 +- util/memory/action.go | 4 +- 9 files changed, 430 insertions(+), 16 deletions(-) create mode 100644 domain/info.go diff --git a/ddl/ddl.go b/ddl/ddl.go index de0a0894359a9..4a66300a5102d 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -218,6 +218,8 @@ type DDL interface { SchemaSyncer() SchemaSyncer // OwnerManager gets the owner manager. OwnerManager() owner.Manager + // GetID gets the ddl ID. + GetID() string // GetTableMaxRowID gets the max row ID of a normal table or a partition. GetTableMaxRowID(startTS uint64, tbl table.Table) (int64, bool, error) // SetBinlogClient sets the binlog client for DDL worker. It's exported for testing. @@ -433,6 +435,11 @@ func (d *ddl) OwnerManager() owner.Manager { return d.ownerManager } +// GetID implements DDL.GetID interface. +func (d *ddl) GetID() string { + return d.uuid +} + func checkJobMaxInterval(job *model.Job) time.Duration { // The job of adding index takes more time to process. // So it uses the longer time. diff --git a/ddl/syncer.go b/ddl/syncer.go index 9d4da63e2b26f..50b1c97e5a7b2 100644 --- a/ddl/syncer.go +++ b/ddl/syncer.go @@ -103,7 +103,11 @@ func NewSchemaSyncer(etcdCli *clientv3.Client, id string) SchemaSyncer { } } -func (s *schemaVersionSyncer) putKV(ctx context.Context, retryCnt int, key, val string, +// PutKVToEtcd puts key value to etcd. +// etcdCli is client of etcd. +// retryCnt is retry time when an error occurs. +// opts is configures of etcd Operations. +func PutKVToEtcd(ctx context.Context, etcdCli *clientv3.Client, retryCnt int, key, val string, opts ...clientv3.OpOption) error { var err error for i := 0; i < retryCnt; i++ { @@ -112,12 +116,12 @@ func (s *schemaVersionSyncer) putKV(ctx context.Context, retryCnt int, key, val } childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) - _, err = s.etcdCli.Put(childCtx, key, val, opts...) + _, err = etcdCli.Put(childCtx, key, val, opts...) cancel() if err == nil { return nil } - log.Warnf("[syncer] put schema version %s failed %v no.%d", val, err, i) + log.Warnf("[etcd-cli] put key: %s value: %s failed %v no.%d", key, val, err, i) time.Sleep(keyOpRetryInterval) } return errors.Trace(err) @@ -148,7 +152,7 @@ func (s *schemaVersionSyncer) Init(ctx context.Context) error { s.mu.globalVerCh = s.etcdCli.Watch(ctx, DDLGlobalSchemaVersion) s.mu.Unlock() - err = s.putKV(ctx, keyOpDefaultRetryCnt, s.selfSchemaVerPath, InitialVersion, + err = PutKVToEtcd(ctx, s.etcdCli, keyOpDefaultRetryCnt, s.selfSchemaVerPath, InitialVersion, clientv3.WithLease(s.session.Lease())) return errors.Trace(err) } @@ -176,7 +180,7 @@ func (s *schemaVersionSyncer) Restart(ctx context.Context) error { childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) defer cancel() - err = s.putKV(childCtx, putKeyRetryUnlimited, s.selfSchemaVerPath, InitialVersion, + err = PutKVToEtcd(childCtx, s.etcdCli, putKeyRetryUnlimited, s.selfSchemaVerPath, InitialVersion, clientv3.WithLease(s.session.Lease())) return errors.Trace(err) @@ -214,7 +218,7 @@ func (s *schemaVersionSyncer) WatchGlobalSchemaVer(ctx context.Context) { func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, version int64) error { startTime := time.Now() ver := strconv.FormatInt(version, 10) - err := s.putKV(ctx, putKeyNoRetry, s.selfSchemaVerPath, ver, + err := PutKVToEtcd(ctx, s.etcdCli, putKeyNoRetry, s.selfSchemaVerPath, ver, clientv3.WithLease(s.session.Lease())) metrics.UpdateSelfVersionHistogram.WithLabelValues(metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -227,8 +231,7 @@ func (s *schemaVersionSyncer) OwnerUpdateGlobalVersion(ctx context.Context, vers ver := strconv.FormatInt(version, 10) // TODO: If the version is larger than the original global version, we need set the version. // Otherwise, we'd better set the original global version. - err := s.putKV(ctx, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) - + err := PutKVToEtcd(ctx, s.etcdCli, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerUpdateGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } @@ -241,15 +244,22 @@ func (s *schemaVersionSyncer) RemoveSelfVersionPath() error { metrics.DeploySyncerHistogram.WithLabelValues(metrics.SyncerClear, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() + err = DeleteKeyFromEtcd(s.selfSchemaVerPath, s.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + return errors.Trace(err) +} + +// DeleteKeyFromEtcd deletes key value from etcd. +func DeleteKeyFromEtcd(key string, etcdCli *clientv3.Client, retryCnt int, timeout time.Duration) error { + var err error ctx := context.Background() - for i := 0; i < keyOpDefaultRetryCnt; i++ { - childCtx, cancel := context.WithTimeout(ctx, keyOpDefaultTimeout) - _, err = s.etcdCli.Delete(childCtx, s.selfSchemaVerPath) + for i := 0; i < retryCnt; i++ { + childCtx, cancel := context.WithTimeout(ctx, timeout) + _, err = etcdCli.Delete(childCtx, key) cancel() if err == nil { return nil } - log.Warnf("[syncer] remove schema version path %s failed %v no.%d", s.selfSchemaVerPath, err, i) + log.Warnf("[etcd-cli] delete key %s failed %v no.%d", key, err, i) } return errors.Trace(err) } @@ -283,7 +293,7 @@ func (s *schemaVersionSyncer) MustGetGlobalVersion(ctx context.Context) (int64, if err != nil { continue } - if err == nil && len(resp.Kvs) > 0 { + if len(resp.Kvs) > 0 { var ver int ver, err = strconv.Atoi(string(resp.Kvs[0].Value)) if err == nil { diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index 5915e29445701..819c7c1933c0a 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -96,6 +96,18 @@ timezone.* curl http://{TiDBIP}:10080/settings ``` +1. Get TiDB server information. + + ```shell + curl http://{TiDBIP}:10080/info + ``` + +1. Get TiDB cluster all servers information. + + ```shell + curl http://{TiDBIP}:10080/info/all + ``` + 1. Enable/Disable TiDB server general log ```shell diff --git a/domain/domain.go b/domain/domain.go index 3007b8afbb397..7f80aa0baf5f0 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -52,6 +52,7 @@ type Domain struct { statsHandle unsafe.Pointer statsLease time.Duration ddl ddl.DDL + info *InfoSyncer m sync.Mutex SchemaValidator SchemaValidator sysSessionPool *pools.ResourcePool @@ -251,6 +252,11 @@ func (do *Domain) DDL() ddl.DDL { return do.ddl } +// InfoSyncer gets infoSyncer from domain. +func (do *Domain) InfoSyncer() *InfoSyncer { + return do.info +} + // Store gets KV store from domain. func (do *Domain) Store() kv.Storage { return do.store @@ -359,6 +365,8 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) { break } do.SchemaValidator.Restart() + case <-do.info.Done(): + do.info.Restart(context.Background()) case <-do.exit: return } @@ -392,6 +400,9 @@ func (do *Domain) Close() { if do.ddl != nil { terror.Log(errors.Trace(do.ddl.Stop())) } + if do.info != nil { + do.info.RemoveServerInfo() + } close(do.exit) if do.etcdClient != nil { terror.Log(errors.Trace(do.etcdClient.Close())) @@ -500,6 +511,11 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R if err != nil { return errors.Trace(err) } + do.info = NewInfoSyncer(do.ddl.GetID(), do.etcdClient) + err = do.info.Init(ctx) + if err != nil { + return errors.Trace(err) + } err = do.Reload() if err != nil { return errors.Trace(err) diff --git a/domain/info.go b/domain/info.go new file mode 100644 index 0000000000000..6ab5a5f5d5bdb --- /dev/null +++ b/domain/info.go @@ -0,0 +1,220 @@ +// Copyright 2018 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 domain + +import ( + "encoding/json" + "fmt" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/coreos/etcd/clientv3/concurrency" + "github.com/juju/errors" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/printer" + log "github.com/sirupsen/logrus" + "golang.org/x/net/context" +) + +const ( + // ServerInformationPath store server information such as IP, port and so on. + ServerInformationPath = "/tidb/server/info" + // keyOpDefaultRetryCnt is the default retry count for etcd store. + keyOpDefaultRetryCnt = 2 + // keyOpDefaultTimeout is the default time out for etcd store. + keyOpDefaultTimeout = 1 * time.Second +) + +// InfoSessionTTL is the etcd session's TTL in seconds. It's exported for testing. +var InfoSessionTTL = 1 * 60 + +// InfoSyncer stores server info to etcd when the tidb-server starts and delete when tidb-server shuts down. +type InfoSyncer struct { + etcdCli *clientv3.Client + info *ServerInfo + serverInfoPath string + session *concurrency.Session +} + +// ServerInfo is server static information. +// It will not be updated when tidb-server running. So please only put static information in ServerInfo struct. +type ServerInfo struct { + ServerVersionInfo + ID string `json:"ddl_id"` + IP string `json:"ip"` + Port uint `json:"listening_port"` + StatusPort uint `json:"status_port"` + Lease string `json:"lease"` +} + +// ServerVersionInfo is the server version and git_hash. +type ServerVersionInfo struct { + Version string `json:"version"` + GitHash string `json:"git_hash"` +} + +// NewInfoSyncer return new InfoSyncer. It is exported for testing. +func NewInfoSyncer(id string, etcdCli *clientv3.Client) *InfoSyncer { + return &InfoSyncer{ + etcdCli: etcdCli, + info: getServerInfo(id), + serverInfoPath: fmt.Sprintf("%s/%s", ServerInformationPath, id), + } +} + +// Init creates a new etcd session and stores server info to etcd. +func (is *InfoSyncer) Init(ctx context.Context) error { + return errors.Trace(is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt)) +} + +// GetServerInfo gets self server static information. +func (is *InfoSyncer) GetServerInfo() *ServerInfo { + return is.info +} + +// GetServerInfoByID gets server static information from etcd. +func (is *InfoSyncer) GetServerInfoByID(ctx context.Context, id string) (*ServerInfo, error) { + if is.etcdCli == nil || id == is.info.ID { + return is.info, nil + } + key := fmt.Sprintf("%s/%s", ServerInformationPath, id) + infoMap, err := getInfo(ctx, is.etcdCli, key, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + if err != nil { + return nil, errors.Trace(err) + } + info, ok := infoMap[id] + if !ok { + return nil, errors.Errorf("[info-syncer] get %s failed", key) + } + return info, nil +} + +// GetAllServerInfo gets all servers static information from etcd. +func (is *InfoSyncer) GetAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { + allInfo := make(map[string]*ServerInfo) + if is.etcdCli == nil { + allInfo[is.info.ID] = is.info + return allInfo, nil + } + allInfo, err := getInfo(ctx, is.etcdCli, ServerInformationPath, keyOpDefaultRetryCnt, keyOpDefaultTimeout, clientv3.WithPrefix()) + if err != nil { + return nil, errors.Trace(err) + } + return allInfo, nil +} + +// storeServerInfo stores self server static information to etcd. +func (is *InfoSyncer) storeServerInfo(ctx context.Context) error { + if is.etcdCli == nil { + return nil + } + infoBuf, err := json.Marshal(is.info) + if err != nil { + return errors.Trace(err) + } + err = ddl.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.serverInfoPath, hack.String(infoBuf), clientv3.WithLease(is.session.Lease())) + return errors.Trace(err) +} + +// RemoveServerInfo remove self server static information from etcd. +func (is *InfoSyncer) RemoveServerInfo() { + if is.etcdCli == nil { + return + } + err := ddl.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + if err != nil { + log.Errorf("[info-syncer] remove server info failed %v", err) + } +} + +// Done returns a channel that closes when the info syncer is no longer being refreshed. +func (is InfoSyncer) Done() <-chan struct{} { + if is.etcdCli == nil { + return make(chan struct{}, 1) + } + return is.session.Done() +} + +// Restart restart the info syncer with new session leaseID and store server info to etcd again. +func (is *InfoSyncer) Restart(ctx context.Context) error { + return errors.Trace(is.newSessionAndStoreServerInfo(ctx, owner.NewSessionDefaultRetryCnt)) +} + +// newSessionAndStoreServerInfo creates a new etcd session and stores server info to etcd. +func (is *InfoSyncer) newSessionAndStoreServerInfo(ctx context.Context, retryCnt int) error { + if is.etcdCli == nil { + return nil + } + var err error + logPrefix := fmt.Sprintf("[Info-syncer] %s", is.serverInfoPath) + is.session, err = owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, InfoSessionTTL) + if err != nil { + return errors.Trace(err) + } + err = is.storeServerInfo(ctx) + return errors.Trace(err) +} + +// getInfo gets server information from etcd according to the key and opts. +func getInfo(ctx context.Context, etcdCli *clientv3.Client, key string, retryCnt int, timeout time.Duration, opts ...clientv3.OpOption) (map[string]*ServerInfo, error) { + var err error + var resp *clientv3.GetResponse + allInfo := make(map[string]*ServerInfo) + for i := 0; i < retryCnt; i++ { + select { + case <-ctx.Done(): + err = errors.Trace(ctx.Err()) + return nil, err + default: + } + childCtx, cancel := context.WithTimeout(ctx, timeout) + resp, err = etcdCli.Get(childCtx, key, opts...) + cancel() + if err != nil { + log.Infof("[info-syncer] get %s failed %v, continue checking.", key, err) + time.Sleep(200 * time.Millisecond) + continue + } + for _, kv := range resp.Kvs { + info := &ServerInfo{} + err = json.Unmarshal(kv.Value, info) + if err != nil { + log.Infof("[info-syncer] get %s, json.Unmarshal %v failed %v.", kv.Key, kv.Value, err) + return nil, errors.Trace(err) + } + allInfo[info.ID] = info + } + return allInfo, nil + } + return nil, errors.Trace(err) +} + +// getServerInfo gets self tidb server information. +func getServerInfo(id string) *ServerInfo { + cfg := config.GetGlobalConfig() + info := &ServerInfo{ + ID: id, + IP: cfg.AdvertiseAddress, + Port: cfg.Port, + StatusPort: cfg.Status.StatusPort, + Lease: cfg.Lease, + } + info.Version = mysql.ServerVersion + info.GitHash = printer.TiDBGitHash + return info +} diff --git a/server/http_handler.go b/server/http_handler.go index 6ac958a3faf3d..e3b2551968a60 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -334,6 +334,14 @@ type ddlHistoryJobHandler struct { *tikvHandlerTool } +type serverInfoHandler struct { + *tikvHandlerTool +} + +type allServerInfoHandler struct { + *tikvHandlerTool +} + // valueHandle is the handler for get value. type valueHandler struct { } @@ -1249,3 +1257,78 @@ func (h *mvccTxnHandler) handleMvccGetByTxn(params map[string]string) (interface } return h.getMvccByStartTs(uint64(startTS), startKey, endKey) } + +// serverInfo is used to report the servers info when do http request. +type serverInfo struct { + IsOwner bool `json:"is_owner"` + *domain.ServerInfo +} + +// ServeHTTP handles request of ddl server info. +func (h serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + do, err := session.GetDomain(h.store.(kv.Storage)) + if err != nil { + writeError(w, errors.New("create session error")) + log.Error(err) + return + } + info := serverInfo{} + info.ServerInfo = do.InfoSyncer().GetServerInfo() + info.IsOwner = do.DDL().OwnerManager().IsOwner() + writeData(w, info) +} + +// clusterServerInfo is used to report cluster servers info when do http request. +type clusterServerInfo struct { + ServersNum int `json:"servers_num,omitempty"` + OwnerID string `json:"owner_id"` + IsAllServerVersionConsistent bool `json:"is_all_server_version_consistent,omitempty"` + AllServersDiffVersions []domain.ServerVersionInfo `json:"all_servers_diff_versions,omitempty"` + AllServersInfo map[string]*domain.ServerInfo `json:"all_servers_info,omitempty"` +} + +// ServeHTTP handles request of all ddl servers info. +func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + do, err := session.GetDomain(h.store.(kv.Storage)) + if err != nil { + writeError(w, errors.New("create session error")) + log.Error(err) + return + } + ctx := context.Background() + allServersInfo, err := do.InfoSyncer().GetAllServerInfo(ctx) + if err != nil { + writeError(w, errors.New("ddl server information not found")) + log.Error(err) + return + } + ctx, cancel := context.WithTimeout(ctx, 3*time.Second) + ownerID, err := do.DDL().OwnerManager().GetOwnerID(ctx) + cancel() + if err != nil { + writeError(w, errors.New("ddl server information not found")) + log.Error(err) + return + } + allVersionsMap := map[domain.ServerVersionInfo]struct{}{} + allVersions := []domain.ServerVersionInfo{} + for _, v := range allServersInfo { + if _, ok := allVersionsMap[v.ServerVersionInfo]; ok { + continue + } + allVersionsMap[v.ServerVersionInfo] = struct{}{} + allVersions = append(allVersions, v.ServerVersionInfo) + } + clusterInfo := clusterServerInfo{ + ServersNum: len(allServersInfo), + OwnerID: ownerID, + // len(allVersions) = 1 indicates there has only 1 tidb version in cluster, so all server versions are consistent. + IsAllServerVersionConsistent: len(allVersions) == 1, + AllServersInfo: allServersInfo, + } + // if IsAllServerVersionConsistent is false, return the all tidb servers version. + if !clusterInfo.IsAllServerVersionConsistent { + clusterInfo.AllServersDiffVersions = allVersions + } + writeData(w, clusterInfo) +} diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 3404055719e82..0dc9b279a7100 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/printer" log "github.com/sirupsen/logrus" ) @@ -645,3 +646,64 @@ func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { } log.Fatalf("Failed to get profile for %d retries in every 10 ms", retryTime) } + +func (ts *HTTPHandlerTestSuite) TestServerInfo(c *C) { + ts.startServer(c) + defer ts.stopServer(c) + resp, err := http.Get("http://127.0.0.1:10090/info") + c.Assert(err, IsNil) + defer resp.Body.Close() + c.Assert(resp.StatusCode, Equals, http.StatusOK) + decoder := json.NewDecoder(resp.Body) + + info := serverInfo{} + err = decoder.Decode(&info) + c.Assert(err, IsNil) + + cfg := config.GetGlobalConfig() + c.Assert(info.IsOwner, IsTrue) + c.Assert(info.IP, Equals, cfg.AdvertiseAddress) + c.Assert(info.StatusPort, Equals, cfg.Status.StatusPort) + c.Assert(info.Lease, Equals, cfg.Lease) + c.Assert(info.Version, Equals, mysql.ServerVersion) + c.Assert(info.GitHash, Equals, printer.TiDBGitHash) + + store := ts.server.newTikvHandlerTool().store.(kv.Storage) + do, err := session.GetDomain(store.(kv.Storage)) + c.Assert(err, IsNil) + ddl := do.DDL() + c.Assert(info.ID, Equals, ddl.GetID()) +} + +func (ts *HTTPHandlerTestSuite) TestAllServerInfo(c *C) { + ts.startServer(c) + defer ts.stopServer(c) + resp, err := http.Get("http://127.0.0.1:10090/info/all") + c.Assert(err, IsNil) + defer resp.Body.Close() + c.Assert(resp.StatusCode, Equals, http.StatusOK) + decoder := json.NewDecoder(resp.Body) + + clusterInfo := clusterServerInfo{} + err = decoder.Decode(&clusterInfo) + c.Assert(err, IsNil) + + c.Assert(clusterInfo.IsAllServerVersionConsistent, IsTrue) + c.Assert(clusterInfo.ServersNum, Equals, 1) + + store := ts.server.newTikvHandlerTool().store.(kv.Storage) + do, err := session.GetDomain(store.(kv.Storage)) + c.Assert(err, IsNil) + ddl := do.DDL() + c.Assert(clusterInfo.OwnerID, Equals, ddl.GetID()) + serverInfo, ok := clusterInfo.AllServersInfo[ddl.GetID()] + c.Assert(ok, Equals, true) + + cfg := config.GetGlobalConfig() + c.Assert(serverInfo.IP, Equals, cfg.AdvertiseAddress) + c.Assert(serverInfo.StatusPort, Equals, cfg.Status.StatusPort) + c.Assert(serverInfo.Lease, Equals, cfg.Lease) + c.Assert(serverInfo.Version, Equals, mysql.ServerVersion) + c.Assert(serverInfo.GitHash, Equals, printer.TiDBGitHash) + c.Assert(serverInfo.ID, Equals, ddl.GetID()) +} diff --git a/server/http_status.go b/server/http_status.go index 0c6012fa31b49..357594d3cb695 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -52,8 +52,12 @@ func (s *Server) startHTTPServer() { router.Handle("/schema/{db}/{table}", schemaHandler{tikvHandlerTool}) router.Handle("/tables/{colID}/{colTp}/{colFlag}/{colLen}", valueHandler{}) router.Handle("/ddl/history", ddlHistoryJobHandler{tikvHandlerTool}) + + // HTTP path for get server info. + router.Handle("/info", serverInfoHandler{tikvHandlerTool}) + router.Handle("/info/all", allServerInfoHandler{tikvHandlerTool}) if s.cfg.Store == "tikv" { - // HTTP path for tikv + // HTTP path for tikv. router.Handle("/tables/{db}/{table}/regions", tableHandler{tikvHandlerTool, opTableRegions}) router.Handle("/tables/{db}/{table}/scatter", tableHandler{tikvHandlerTool, opTableScatter}) router.Handle("/tables/{db}/{table}/stop-scatter", tableHandler{tikvHandlerTool, opStopTableScatter}) diff --git a/util/memory/action.go b/util/memory/action.go index d06b33c22d733..06a2d39deded8 100644 --- a/util/memory/action.go +++ b/util/memory/action.go @@ -45,13 +45,13 @@ func (a *LogOnExceed) Action(t *Tracker) { } } -// PanicOnExceed panics when when memory usage exceeds memory quota. +// PanicOnExceed panics when memory usage exceeds memory quota. type PanicOnExceed struct { mutex sync.Mutex // For synchronization. acted bool } -// Action panics when when memory usage exceeds memory quota. +// Action panics when memory usage exceeds memory quota. func (a *PanicOnExceed) Action(t *Tracker) { a.mutex.Lock() if a.acted { From 29addca22198d6832382fb8a7192d355dbedec92 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 15 Aug 2018 22:01:44 +0800 Subject: [PATCH 60/87] stats: log detailed stats info for query feedback (#7293) --- statistics/boostrap.go | 14 ++++ statistics/feedback.go | 137 +++++++++++++++++++++++++++++++++++- statistics/feedback_test.go | 56 +++++++-------- statistics/handle.go | 1 + statistics/histogram.go | 33 +++++---- statistics/table.go | 2 + statistics/update.go | 10 +++ statistics/update_test.go | 103 +++++++++++++++++++++++---- types/datum.go | 17 +++-- util/codec/codec.go | 37 ++++++++++ 10 files changed, 351 insertions(+), 59 deletions(-) diff --git a/statistics/boostrap.go b/statistics/boostrap.go index 0a6862394a16e..fb28b42cda7d1 100644 --- a/statistics/boostrap.go +++ b/statistics/boostrap.go @@ -14,6 +14,8 @@ package statistics import ( + "fmt" + "github.com/juju/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/model" @@ -49,6 +51,7 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache tbl := &Table{ HistColl: newHistColl, Version: row.GetUint64(0), + name: getFullTableName(is, tableInfo), } tables[physicalID] = tbl } @@ -257,3 +260,14 @@ func (h *Handle) InitStats(is infoschema.InfoSchema) error { h.statsCache.Store(tables) return nil } + +func getFullTableName(is infoschema.InfoSchema, tblInfo *model.TableInfo) string { + for _, schema := range is.AllSchemas() { + if t, err := is.TableByName(schema.Name, tblInfo.Name); err == nil { + if t.Meta().ID == tblInfo.ID { + return schema.Name.O + "." + tblInfo.Name.O + } + } + } + return fmt.Sprintf("%d", tblInfo.ID) +} diff --git a/statistics/feedback.go b/statistics/feedback.go index 451e076878b6d..652b02386f242 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -16,6 +16,7 @@ package statistics import ( "bytes" "encoding/gob" + "fmt" "math" "math/rand" "sort" @@ -102,11 +103,11 @@ func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { if isIndex { var err error // As we do not know the origin length, just use a custom value here. - lowVal, err = codec.Decode(low.GetBytes(), 4) + lowVal, err = codec.DecodeRange(low.GetBytes(), 4) if err != nil { return nil, errors.Trace(err) } - highVal, err = codec.Decode(high.GetBytes(), 4) + highVal, err = codec.DecodeRange(high.GetBytes(), 4) if err != nil { return nil, errors.Trace(err) } @@ -759,3 +760,135 @@ func splitFeedbackByQueryType(feedbacks []feedback) ([]feedback, []feedback) { } return eqFB, ranFB } + +// formatBuckets formats bucket from lowBkt to highBkt. +func formatBuckets(hg *Histogram, lowBkt, highBkt, idxCols int) string { + if lowBkt == highBkt { + return hg.bucketToString(lowBkt, idxCols) + } + if lowBkt+1 == highBkt { + return fmt.Sprintf("%s, %s", hg.bucketToString(lowBkt, 0), hg.bucketToString(highBkt, 0)) + } + // do not care the middle buckets + return fmt.Sprintf("%s, (%d buckets, total count %d), %s", hg.bucketToString(lowBkt, 0), + highBkt-lowBkt-1, hg.Buckets[highBkt-1].Count-hg.Buckets[lowBkt].Count, hg.bucketToString(highBkt, 0)) +} + +func colRangeToStr(c *Column, ran *ranger.Range, actual int64, factor float64) string { + lowCount, lowBkt := c.lessRowCountWithBktIdx(ran.LowVal[0]) + highCount, highBkt := c.lessRowCountWithBktIdx(ran.HighVal[0]) + return fmt.Sprintf("range: %s, actual: %d, expected: %d, buckets: {%s}", ran.String(), actual, + int64((highCount-lowCount)*factor), formatBuckets(&c.Histogram, lowBkt, highBkt, 0)) +} + +func logForPK(prefix string, c *Column, ranges []*ranger.Range, actual []int64, factor float64) { + for i, ran := range ranges { + if ran.LowVal[0].GetInt64()+1 >= ran.HighVal[0].GetInt64() { + continue + } + log.Debugf("%s column: %s, %s", prefix, c.Info.Name, colRangeToStr(c, ran, actual[i], factor)) + } +} + +func logForIndexRange(idx *Index, ran *ranger.Range, actual int64, factor float64) string { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + lb, err := codec.EncodeKey(sc, nil, ran.LowVal...) + if err != nil { + return "" + } + rb, err := codec.EncodeKey(sc, nil, ran.HighVal...) + if err != nil { + return "" + } + if idx.CMSketch != nil && bytes.Compare(kv.Key(lb).PrefixNext(), rb) >= 0 { + str, err := types.DatumsToString(ran.LowVal, true) + if err != nil { + return "" + } + return fmt.Sprintf("value: %s, actual: %d, expected: %d", str, actual, int64(float64(idx.QueryBytes(lb))*factor)) + } + l, r := types.NewBytesDatum(lb), types.NewBytesDatum(rb) + lowCount, lowBkt := idx.lessRowCountWithBktIdx(l) + highCount, highBkt := idx.lessRowCountWithBktIdx(r) + return fmt.Sprintf("range: %s, actual: %d, expected: %d, histogram: {%s}", ran.String(), actual, + int64((highCount-lowCount)*factor), formatBuckets(&idx.Histogram, lowBkt, highBkt, len(idx.Info.Columns))) +} + +func logForIndex(prefix string, t *Table, idx *Index, ranges []*ranger.Range, actual []int64, factor float64) { + sc := &stmtctx.StatementContext{TimeZone: time.UTC} + if idx.CMSketch == nil || idx.statsVer != version1 { + for i, ran := range ranges { + log.Debugf("%s index: %s, %s", prefix, idx.Info.Name.O, logForIndexRange(idx, ran, actual[i], factor)) + } + return + } + for i, ran := range ranges { + rangePosition := getOrdinalOfRangeCond(sc, ran) + // only contains range or equality query + if rangePosition == 0 || rangePosition == len(ran.LowVal) { + log.Debugf("%s index: %s, %s", prefix, idx.Info.Name.O, logForIndexRange(idx, ran, actual[i], factor)) + continue + } + equalityString, err := types.DatumsToString(ran.LowVal[:rangePosition], true) + if err != nil { + continue + } + bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) + if err != nil { + continue + } + equalityCount := idx.CMSketch.QueryBytes(bytes) + rang := ranger.Range{ + LowVal: []types.Datum{ran.LowVal[rangePosition]}, + HighVal: []types.Datum{ran.HighVal[rangePosition]}, + } + colName := idx.Info.Columns[rangePosition].Name.L + var rangeString string + // prefer index stats over column stats + if idx, ok := t.colName2Idx[colName]; ok { + if t.Indices[idx] == nil { + return + } + rangeString = logForIndexRange(t.Indices[idx], &rang, -1, factor) + } else { + id := t.colName2ID[colName] + if t.Columns[id] == nil { + return + } + rangeString = colRangeToStr(t.Columns[t.colName2ID[colName]], &rang, -1, factor) + } + log.Debugf("%s index: %s, actual: %d, equality: %s, expected equality: %d, %s", prefix, idx.Info.Name.O, + actual[i], equalityString, equalityCount, rangeString) + } +} + +func (q *QueryFeedback) logDetailedInfo(h *Handle) { + t, ok := h.statsCache.Load().(statsCache)[q.tableID] + if !ok { + return + } + isIndex := q.hist.isIndexHist() + ranges, err := q.DecodeToRanges(isIndex) + if err != nil { + log.Debug(err) + return + } + actual := make([]int64, 0, len(q.feedback)) + for _, fb := range q.feedback { + actual = append(actual, fb.count) + } + logPrefix := fmt.Sprintf("[stats-feedback] %s,", t.name) + if isIndex { + idx := t.Indices[q.hist.ID] + if idx == nil { + return + } + logForIndex(logPrefix, t, idx, ranges, actual, idx.getIncreaseFactor(t.Count)) + } else { + c := t.Columns[q.hist.ID] + if c == nil { + return + } + logForPK(logPrefix, c, ranges, actual, c.getIncreaseFactor(t.Count)) + } +} diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 17493375a1894..4f78a133c8295 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -71,13 +71,13 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { defer func() { defaultBucketCount = originBucketCount }() c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ - "num: 10000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ - "num: 10008\tlower_bound: 2\tupper_bound: 7\trepeats: 0\n"+ - "num: 10019\tlower_bound: 8\tupper_bound: 19\trepeats: 0\n"+ - "num: 10019\tlower_bound: 20\tupper_bound: 20\trepeats: 0\n"+ - "num: 10037\tlower_bound: 21\tupper_bound: 39\trepeats: 0\n"+ - "num: 10055\tlower_bound: 40\tupper_bound: 58\trepeats: 0\n"+ - "num: 10057\tlower_bound: 59\tupper_bound: 60\trepeats: 0") + "num: 10000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ + "num: 8 lower_bound: 2 upper_bound: 7 repeats: 0\n"+ + "num: 11 lower_bound: 8 upper_bound: 19 repeats: 0\n"+ + "num: 0 lower_bound: 20 upper_bound: 20 repeats: 0\n"+ + "num: 18 lower_bound: 21 upper_bound: 39 repeats: 0\n"+ + "num: 18 lower_bound: 40 upper_bound: 58 repeats: 0\n"+ + "num: 2 lower_bound: 59 upper_bound: 60 repeats: 0") } func (s *testFeedbackSuite) TestSplitBuckets(c *C) { @@ -91,12 +91,12 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { buckets, isNewBuckets, totalCount := splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ - "num: 1\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ - "num: 1\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ - "num: 1\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ - "num: 6\tlower_bound: 10\tupper_bound: 15\trepeats: 0\n"+ - "num: 6\tlower_bound: 16\tupper_bound: 20\trepeats: 0\n"+ - "num: 6\tlower_bound: 30\tupper_bound: 50\trepeats: 0") + "num: 1 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ + "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0\n"+ + "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0\n"+ + "num: 5 lower_bound: 10 upper_bound: 15 repeats: 0\n"+ + "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0\n"+ + "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false, false, false, true, true, false}) c.Assert(totalCount, Equals, int64(6)) @@ -110,12 +110,12 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ - "num: 100000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ - "num: 100000\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ - "num: 100000\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ - "num: 100001\tlower_bound: 10\tupper_bound: 15\trepeats: 0\n"+ - "num: 100001\tlower_bound: 16\tupper_bound: 20\trepeats: 0\n"+ - "num: 100001\tlower_bound: 30\tupper_bound: 50\trepeats: 0") + "num: 100000 lower_bound: 0 upper_bound: 1 repeats: 0\n"+ + "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0\n"+ + "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0\n"+ + "num: 1 lower_bound: 10 upper_bound: 15 repeats: 0\n"+ + "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0\n"+ + "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false, false, false, true, true, false}) c.Assert(totalCount, Equals, int64(100001)) @@ -132,7 +132,7 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ - "num: 1000000\tlower_bound: 0\tupper_bound: 1000000\trepeats: 0") + "num: 1000000 lower_bound: 0 upper_bound: 1000000 repeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false}) c.Assert(totalCount, Equals, int64(1000000)) @@ -148,8 +148,8 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, "column:0 ndv:0 totColSize:0\n"+ - "num: 1\tlower_bound: 0\tupper_bound: 10\trepeats: 0\n"+ - "num: 1\tlower_bound: 11\tupper_bound: 1000000\trepeats: 0") + "num: 1 lower_bound: 0 upper_bound: 10 repeats: 0\n"+ + "num: 0 lower_bound: 11 upper_bound: 1000000 repeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{true, true}) c.Assert(totalCount, Equals, int64(1)) } @@ -169,7 +169,7 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { counts: []int64{1}, isNewBuckets: []bool{false}, bucketCount: 1, - result: "column:0 ndv:0 totColSize:0\nnum: 1\tlower_bound: 1\tupper_bound: 2\trepeats: 0", + result: "column:0 ndv:0 totColSize:0\nnum: 1 lower_bound: 1 upper_bound: 2 repeats: 0", }, { points: []int64{1, 2, 2, 3, 3, 4}, @@ -177,8 +177,8 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { isNewBuckets: []bool{false, false, false}, bucketCount: 2, result: "column:0 ndv:0 totColSize:0\n" + - "num: 100000\tlower_bound: 1\tupper_bound: 2\trepeats: 0\n" + - "num: 100002\tlower_bound: 2\tupper_bound: 4\trepeats: 0", + "num: 100000 lower_bound: 1 upper_bound: 2 repeats: 0\n" + + "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0", }, // test do not merge if the result bucket count is too large { @@ -187,9 +187,9 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { isNewBuckets: []bool{false, false, false, false}, bucketCount: 3, result: "column:0 ndv:0 totColSize:0\n" + - "num: 2\tlower_bound: 1\tupper_bound: 3\trepeats: 0\n" + - "num: 100002\tlower_bound: 3\tupper_bound: 4\trepeats: 0\n" + - "num: 200002\tlower_bound: 4\tupper_bound: 5\trepeats: 0", + "num: 2 lower_bound: 1 upper_bound: 3 repeats: 0\n" + + "num: 100000 lower_bound: 3 upper_bound: 4 repeats: 0\n" + + "num: 100000 lower_bound: 4 upper_bound: 5 repeats: 0", }, } for _, t := range tests { diff --git a/statistics/handle.go b/statistics/handle.go index 9785b2a6ee656..c0ddaec4e0ea6 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -159,6 +159,7 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { tbl.Version = version tbl.Count = count tbl.ModifyCount = modifyCount + tbl.name = getFullTableName(is, tableInfo) tables = append(tables, tbl) } h.mu.Lock() diff --git a/statistics/histogram.go b/statistics/histogram.go index 13195dc15fd89..372a2ff8f96f7 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -341,7 +341,7 @@ func ValueToString(value *types.Datum, idxCols int) (string, error) { if idxCols == 0 { return value.ToString() } - decodedVals, err := codec.Decode(value.GetBytes(), idxCols) + decodedVals, err := codec.DecodeRange(value.GetBytes(), idxCols) if err != nil { return "", errors.Trace(err) } @@ -352,6 +352,14 @@ func ValueToString(value *types.Datum, idxCols int) (string, error) { return str, nil } +func (hg *Histogram) bucketToString(bktID, idxCols int) string { + upperVal, err := ValueToString(hg.GetUpper(bktID), idxCols) + terror.Log(errors.Trace(err)) + lowerVal, err := ValueToString(hg.GetLower(bktID), idxCols) + terror.Log(errors.Trace(err)) + return fmt.Sprintf("num: %d lower_bound: %s upper_bound: %s repeats: %d", hg.bucketCount(bktID), lowerVal, upperVal, hg.Buckets[bktID].Repeat) +} + // ToString gets the string representation for the histogram. func (hg *Histogram) ToString(idxCols int) string { strs := make([]string, 0, hg.Len()+1) @@ -361,11 +369,7 @@ func (hg *Histogram) ToString(idxCols int) string { strs = append(strs, fmt.Sprintf("column:%d ndv:%d totColSize:%d", hg.ID, hg.NDV, hg.TotColSize)) } for i := 0; i < hg.Len(); i++ { - upperVal, err := ValueToString(hg.GetUpper(i), idxCols) - terror.Log(errors.Trace(err)) - lowerVal, err := ValueToString(hg.GetLower(i), idxCols) - terror.Log(errors.Trace(err)) - strs = append(strs, fmt.Sprintf("num: %d\tlower_bound: %s\tupper_bound: %s\trepeats: %d", hg.Buckets[i].Count, lowerVal, upperVal, hg.Buckets[i].Repeat)) + strs = append(strs, hg.bucketToString(i, idxCols)) } return strings.Join(strs, "\n") } @@ -405,14 +409,14 @@ func (hg *Histogram) greaterAndEqRowCount(value types.Datum) float64 { } // lessRowCount estimates the row count where the column less than value. -func (hg *Histogram) lessRowCount(value types.Datum) float64 { +func (hg *Histogram) lessRowCountWithBktIdx(value types.Datum) (float64, int) { // all the values is null if hg.Bounds == nil { - return 0 + return 0, 0 } index, match := hg.Bounds.LowerBound(0, &value) if index == hg.Bounds.NumRows() { - return hg.totalRowCount() + return hg.totalRowCount(), hg.Len() - 1 } // Since we store the lower and upper bound together, so dividing the index by 2 will get the bucket index. bucketIdx := index / 2 @@ -423,11 +427,16 @@ func (hg *Histogram) lessRowCount(value types.Datum) float64 { } if index%2 == 1 { if match { - return curCount - curRepeat + return curCount - curRepeat, bucketIdx } - return preCount + hg.calcFraction(bucketIdx, &value)*(curCount-curRepeat-preCount) + return preCount + hg.calcFraction(bucketIdx, &value)*(curCount-curRepeat-preCount), bucketIdx } - return preCount + return preCount, bucketIdx +} + +func (hg *Histogram) lessRowCount(value types.Datum) float64 { + result, _ := hg.lessRowCountWithBktIdx(value) + return result } // lessAndEqRowCount estimates the row count where the column less than or equal to value. diff --git a/statistics/table.go b/statistics/table.go index 7ab12882370ae..e6d23b13a87fd 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -46,6 +46,7 @@ const ( type Table struct { HistColl Version uint64 + name string } // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. @@ -88,6 +89,7 @@ func (t *Table) copy() *Table { nt := &Table{ HistColl: newHistColl, Version: t.Version, + name: t.name, } return nt } diff --git a/statistics/update.go b/statistics/update.go index 6d8631d6d1903..fa4f73234369c 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -163,6 +163,13 @@ func mergeQueryFeedback(lq []*QueryFeedback, rq []*QueryFeedback) []*QueryFeedba return lq } +var ( + // MinLogScanCount is the minimum scan count for a feedback to be logged. + MinLogScanCount = int64(1000) + // MinLogErrorRate is the minimum error rate for a feedback to be logged. + MinLogErrorRate = 0.5 +) + // StoreQueryFeedback will merges the feedback into stats collector. func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle) error { q := feedback.(*QueryFeedback) @@ -185,6 +192,9 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand } else { rate = math.Abs(expected-float64(q.actual)) / float64(q.actual) } + if rate >= MinLogErrorRate && (q.actual >= MinLogScanCount || q.expected >= MinLogScanCount) { + q.logDetailedInfo(h) + } metrics.StatsInaccuracyRate.Observe(rate) s.Lock() defer s.Unlock() diff --git a/statistics/update_test.go b/statistics/update_test.go index 651d1e698268f..5d41462039908 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -20,6 +20,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" + log "github.com/sirupsen/logrus" ) var _ = Suite(&testStatsUpdateSuite{}) @@ -555,25 +557,25 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { // test primary key feedback sql: "select * from t where t.a <= 5", hist: "column:1 ndv:3 totColSize:0\n" + - "num: 1\tlower_bound: -9223372036854775808\tupper_bound: 1\trepeats: 0\n" + - "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 1\n" + - "num: 4\tlower_bound: 3\tupper_bound: 5\trepeats: 0", + "num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" + + "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n" + + "num: 2 lower_bound: 3 upper_bound: 5 repeats: 0", idxCols: 0, }, { // test index feedback by double read sql: "select * from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + - "num: 2\tlower_bound: \tupper_bound: 2\trepeats: 0\n" + - "num: 4\tlower_bound: 3\tupper_bound: 6\trepeats: 0", + "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n" + + "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0", idxCols: 1, }, { // test index feedback by single read sql: "select b from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + - "num: 2\tlower_bound: \tupper_bound: 2\trepeats: 0\n" + - "num: 4\tlower_bound: 3\tupper_bound: 6\trepeats: 0", + "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n" + + "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0", idxCols: 1, }, } @@ -710,10 +712,9 @@ func (s *testStatsUpdateSuite) TestUpdateStatsByLocalFeedback(c *C) { tbl = h.GetTableStats(tblInfo) c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, "column:1 ndv:3 totColSize:0\n"+ - "num: 1\tlower_bound: 1\tupper_bound: 1\trepeats: 1\n"+ - "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 1\n"+ - "num: 4\tlower_bound: 3\tupper_bound: 9223372036854775807\trepeats: 0") - + "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1\n"+ + "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1\n"+ + "num: 2 lower_bound: 3 upper_bound: 9223372036854775807 repeats: 0") sc := &stmtctx.StatementContext{TimeZone: time.Local} low, err := codec.EncodeKey(sc, nil, types.NewIntDatum(5)) c.Assert(err, IsNil) @@ -721,9 +722,85 @@ func (s *testStatsUpdateSuite) TestUpdateStatsByLocalFeedback(c *C) { c.Assert(tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low), Equals, uint32(2)) c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, "index:1 ndv:2\n"+ - "num: 2\tlower_bound: \tupper_bound: 2\trepeats: 0\n"+ - "num: 4\tlower_bound: 3\tupper_bound: 6\trepeats: 0") + "num: 2 lower_bound: -inf upper_bound: 2 repeats: 0\n"+ + "num: 2 lower_bound: 3 upper_bound: 6 repeats: 0") // Test that it won't cause panic after update. testKit.MustQuery("select * from t use index(idx) where b > 0") } + +type logHook struct { + results string +} + +func (hook *logHook) Levels() []log.Level { + return []log.Level{log.DebugLevel} +} + +func (hook *logHook) Fire(entry *log.Entry) error { + message := entry.Message + if idx := strings.Index(message, "[stats"); idx != -1 { + hook.results = hook.results + message[idx:] + } + return nil +} + +func (s *testStatsUpdateSuite) TestLogDetailedInfo(c *C) { + defer cleanEnv(c, s.store, s.do) + + oriProbability := statistics.FeedbackProbability + oriMinLogCount := statistics.MinLogScanCount + oriMinError := statistics.MinLogErrorRate + oriLevel := log.GetLevel() + oriBucketNum := executor.GetMaxBucketSizeForTest() + defer func() { + statistics.FeedbackProbability = oriProbability + statistics.MinLogScanCount = oriMinLogCount + statistics.MinLogErrorRate = oriMinError + executor.SetMaxBucketSizeForTest(oriBucketNum) + log.SetLevel(oriLevel) + }() + executor.SetMaxBucketSizeForTest(4) + statistics.FeedbackProbability = 1 + statistics.MinLogScanCount = 0 + statistics.MinLogErrorRate = 0 + + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b), index idx_ba(b,a))") + for i := 0; i < 20; i++ { + testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + testKit.MustExec("analyze table t") + tests := []struct { + sql string + result string + }{ + { + sql: "select * from t where t.a <= 15", + result: "[stats-feedback] test.t, column: a, range: [-inf,7), actual: 8, expected: 7, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}" + + "[stats-feedback] test.t, column: a, range: [8,15), actual: 8, expected: 7, buckets: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1}", + }, + { + sql: "select * from t use index(idx) where t.b <= 15", + result: "[stats-feedback] test.t, index: idx, range: [-inf,7), actual: 8, expected: 7, histogram: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}" + + "[stats-feedback] test.t, index: idx, range: [8,15), actual: 8, expected: 7, histogram: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1}", + }, + { + sql: "select b from t use index(idx_ba) where b = 1 and a <= 5", + result: "[stats-feedback] test.t, index: idx_ba, actual: 1, equality: 1, expected equality: 1, range: [-inf,6], actual: -1, expected: 6, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1}", + }, + { + sql: "select b from t use index(idx_ba) where b = 1", + result: "[stats-feedback] test.t, index: idx_ba, value: 1, actual: 1, expected: 1", + }, + } + log.SetLevel(log.DebugLevel) + var hook logHook + log.AddHook(&hook) + for _, t := range tests { + hook.results = "" + testKit.MustQuery(t.sql) + c.Assert(hook.results, Equals, t.result) + } +} diff --git a/types/datum.go b/types/datum.go index 228e91d3dc45f..cf4e886a0f260 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1827,12 +1827,21 @@ func handleTruncateError(sc *stmtctx.StatementContext) error { } // DatumsToString converts several datums to formatted string. -func DatumsToString(datums []Datum, handleNULL bool) (string, error) { +func DatumsToString(datums []Datum, handleSpecialValue bool) (string, error) { var strs []string for _, datum := range datums { - if datum.Kind() == KindNull && handleNULL { - strs = append(strs, "NULL") - continue + if handleSpecialValue { + switch datum.Kind() { + case KindNull: + strs = append(strs, "NULL") + continue + case KindMinNotNull: + strs = append(strs, "-inf") + continue + case KindMaxValue: + strs = append(strs, "+inf") + continue + } } str, err := datum.ToString() if err != nil { diff --git a/util/codec/codec.go b/util/codec/codec.go index 57b462f3e359d..1af3817445635 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -311,6 +311,43 @@ func Decode(b []byte, size int) ([]types.Datum, error) { return values, nil } +// DecodeRange decodes the range values from a byte slice that generated by EncodeKey. +// It handles some special values like `MinNotNull` and `MaxValueDatum`. +func DecodeRange(b []byte, size int) ([]types.Datum, error) { + if len(b) < 1 { + return nil, errors.New("invalid encoded key: length of key is zero") + } + + var ( + err error + values = make([]types.Datum, 0, size) + ) + + for len(b) > 1 { + var d types.Datum + b, d, err = DecodeOne(b) + if err != nil { + return nil, errors.Trace(err) + } + values = append(values, d) + } + + if len(b) == 1 { + switch b[0] { + case NilFlag: + values = append(values, types.Datum{}) + case bytesFlag: + values = append(values, types.MinNotNullDatum()) + // `maxFlag + 1` for PrefixNext + case maxFlag, maxFlag + 1: + values = append(values, types.MaxValueDatum()) + default: + return nil, errors.Errorf("invalid encoded key flag %v", b[0]) + } + } + return values, nil +} + // DecodeOne decodes on datum from a byte slice generated with EncodeKey or EncodeValue. func DecodeOne(b []byte) (remain []byte, d types.Datum, err error) { if len(b) < 1 { From 1f8710762dfeb22829c46984d88d4addc913e7c4 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 16 Aug 2018 00:02:45 +0800 Subject: [PATCH 61/87] execute/insert: refine data truncate error msg (#7401) --- executor/executor_test.go | 5 +++-- executor/insert_common.go | 16 ++++++++++------ executor/insert_test.go | 11 +++++++++++ table/table.go | 2 ++ 4 files changed, 26 insertions(+), 8 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index cc97198beebef..2813abf9bb01a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" @@ -2073,9 +2074,9 @@ func (s *testSuite) TestEmptyEnum(c *C) { tk.MustExec("create table t (e enum('Y', 'N'))") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") _, err := tk.Exec("insert into t values (0)") - c.Assert(terror.ErrorEqual(err, types.ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) _, err = tk.Exec("insert into t values ('abc')") - c.Assert(terror.ErrorEqual(err, types.ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) tk.MustExec("set sql_mode=''") tk.MustExec("insert into t values (0)") diff --git a/executor/insert_common.go b/executor/insert_common.go index b2e8c9d3989d7..99b4ca6d1f50e 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -165,7 +165,7 @@ func (e *InsertValues) insertRows(cols []*table.Column, exec func(rows [][]types return errors.Trace(exec(rows)) } -func (e *InsertValues) handleErr(col *table.Column, rowIdx int, err error) error { +func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int, err error) error { if err == nil { return nil } @@ -175,7 +175,11 @@ func (e *InsertValues) handleErr(col *table.Column, rowIdx int, err error) error } if types.ErrOverflow.Equal(err) { - return types.ErrWarnDataOutOfRange.GenByArgs(col.Name.O, int64(rowIdx+1)) + return types.ErrWarnDataOutOfRange.GenByArgs(col.Name.O, rowIdx+1) + } + if types.ErrTruncated.Equal(err) { + valStr, _ := val.ToString() + return table.ErrTruncatedWrongValueForField.GenByArgs(types.TypeStr(col.Tp), valStr, col.Name.O, rowIdx+1) } return e.filterErr(err) } @@ -198,16 +202,16 @@ func (e *InsertValues) getRow(cols []*table.Column, list []expression.Expression for i, expr := range list { val, err := expr.Eval(chunk.MutRowFromDatums(row).ToRow()) - if err = e.handleErr(cols[i], rowIdx, err); err != nil { + if err = e.handleErr(cols[i], &val, rowIdx, err); err != nil { return nil, errors.Trace(err) } - val, err = table.CastValue(e.ctx, val, cols[i].ToInfo()) - if err = e.handleErr(cols[i], rowIdx, err); err != nil { + val1, err := table.CastValue(e.ctx, val, cols[i].ToInfo()) + if err = e.handleErr(cols[i], &val, rowIdx, err); err != nil { return nil, errors.Trace(err) } offset := cols[i].Offset - row[offset], hasValue[offset] = val, true + row[offset], hasValue[offset] = val1, true } return e.fillGenColData(cols, len(list), hasValue, row) diff --git a/executor/insert_test.go b/executor/insert_test.go index 6b82a4064d7c4..6d928ae1420c5 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -15,6 +15,8 @@ package executor_test import ( . "github.com/pingcap/check" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/testkit" ) @@ -80,3 +82,12 @@ func (s *testSuite) TestInsertOnDuplicateKey(c *C) { tk.MustExec(`insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2;`) tk.MustQuery(`select * from t1`).Check(testkit.Rows("1 400")) } + +func (s *testSuite) TestInsertWrongValueForField(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists t1;`) + tk.MustExec(`create table t1(a bigint);`) + _, err := tk.Exec(`insert into t1 values("asfasdfsajhlkhlksdaf");`) + c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) +} diff --git a/table/table.go b/table/table.go index 300fa9a83045c..e268c7c08c156 100644 --- a/table/table.go +++ b/table/table.go @@ -77,6 +77,8 @@ var ( ErrInvalidRecordKey = terror.ClassTable.New(codeInvalidRecordKey, "invalid record key") // ErrTruncateWrongValue returns for truncate wrong value for field. ErrTruncateWrongValue = terror.ClassTable.New(codeTruncateWrongValue, "incorrect value") + // ErrTruncatedWrongValueForField returns for truncate wrong value for field. + ErrTruncatedWrongValueForField = terror.ClassTable.New(codeTruncateWrongValue, mysql.MySQLErrName[mysql.ErrTruncatedWrongValueForField]) // ErrTrgInvalidCreationCtx happens when inserting a value outside the table partitions. ErrTrgInvalidCreationCtx = terror.ClassTable.New(codeTrgInvalidCreationCtx, "locate partition failed") ) From 82a6c1085ce25962278b73d7844062a76b64c830 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 16 Aug 2018 10:33:24 +0800 Subject: [PATCH 62/87] *: introduce the concept of physical ID, code refactory (#7374) --- ddl/db_test.go | 2 +- ddl/ddl.go | 2 +- ddl/delete_range.go | 18 +++--- ddl/index.go | 46 ++++++++-------- ddl/partition.go | 10 ++-- ddl/reorg.go | 40 +++++++------- executor/analyze.go | 52 +++++++++--------- executor/builder.go | 94 ++++++++++++++++---------------- executor/distsql.go | 48 ++++++++-------- executor/table_reader.go | 14 ++--- infoschema/tables.go | 2 +- meta/meta.go | 22 ++++---- perfschema/tables.go | 2 +- plan/common_plans.go | 11 ++-- plan/find_best_task.go | 22 ++++---- plan/logical_plans.go | 4 +- plan/physical_plans.go | 12 ++-- plan/planbuilder.go | 8 +-- plan/rule_partition_processor.go | 2 +- plan/stringer.go | 2 +- statistics/builder.go | 13 +++-- table/table.go | 18 +++--- table/tables/index.go | 5 +- table/tables/partition.go | 12 ++-- table/tables/tables.go | 37 ++++++------- table/tables/tables_test.go | 6 +- 26 files changed, 252 insertions(+), 252 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index e134ddadbc615..edcfa89b8f96a 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2306,7 +2306,7 @@ func (s *testDBSuite) getMaxTableRowID(ctx *testMaxTableRowIDContext) (int64, bo tbl := ctx.tbl curVer, err := s.store.CurrentVersion() c.Assert(err, IsNil) - maxID, emptyTable, err := d.GetTableMaxRowID(curVer.Ver, tbl) + maxID, emptyTable, err := d.GetTableMaxRowID(curVer.Ver, tbl.(table.PhysicalTable)) c.Assert(err, IsNil) return maxID, emptyTable } diff --git a/ddl/ddl.go b/ddl/ddl.go index 4a66300a5102d..280629d9baeab 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -221,7 +221,7 @@ type DDL interface { // GetID gets the ddl ID. GetID() string // GetTableMaxRowID gets the max row ID of a normal table or a partition. - GetTableMaxRowID(startTS uint64, tbl table.Table) (int64, bool, error) + GetTableMaxRowID(startTS uint64, tbl table.PhysicalTable) (int64, bool, error) // SetBinlogClient sets the binlog client for DDL worker. It's exported for testing. SetBinlogClient(interface{}) } diff --git a/ddl/delete_range.go b/ddl/delete_range.go index ac87c044df7a1..648168ee15afe 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -239,12 +239,12 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error tableID := job.TableID // The startKey here is for compatibility with previous versions, old version did not endKey so don't have to deal with. var startKey kv.Key - var partitionIDs []int64 - if err := job.DecodeArgs(startKey, &partitionIDs); err != nil { + var physicalTableIDs []int64 + if err := job.DecodeArgs(startKey, &physicalTableIDs); err != nil { return errors.Trace(err) } - if len(partitionIDs) > 0 { - for _, pid := range partitionIDs { + if len(physicalTableIDs) > 0 { + for _, pid := range physicalTableIDs { startKey = tablecodec.EncodeTablePrefix(pid) endKey := tablecodec.EncodeTablePrefix(pid + 1) if err := doInsert(s, job.ID, pid, startKey, endKey, now); err != nil { @@ -258,13 +258,13 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error endKey := tablecodec.EncodeTablePrefix(tableID + 1) return doInsert(s, job.ID, tableID, startKey, endKey, now) case model.ActionDropTablePartition: - var partitionID int64 - if err := job.DecodeArgs(&partitionID); err != nil { + var physicalTableID int64 + if err := job.DecodeArgs(&physicalTableID); err != nil { return errors.Trace(err) } - startKey := tablecodec.EncodeTablePrefix(partitionID) - endKey := tablecodec.EncodeTablePrefix(partitionID + 1) - return doInsert(s, job.ID, partitionID, startKey, endKey, now) + startKey := tablecodec.EncodeTablePrefix(physicalTableID) + endKey := tablecodec.EncodeTablePrefix(physicalTableID + 1) + return doInsert(s, job.ID, physicalTableID, startKey, endKey, now) // ActionAddIndex needs do it, because it needs to be rolled back when it's canceled. case model.ActionAddIndex: tableID := job.TableID diff --git a/ddl/index.go b/ddl/index.go index 623bc8b7f313b..7e9e3b7c5be62 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -480,9 +480,9 @@ type addIndexWorker struct { } type reorgIndexTask struct { - partitionID int64 - startHandle int64 - endHandle int64 + physicalTableID int64 + startHandle int64 + endHandle int64 // endIncluded indicates whether the range include the endHandle. // When the last handle is math.MaxInt64, set endIncluded to true to // tell worker backfilling index of endHandle. @@ -496,8 +496,8 @@ type addIndexResult struct { err error } -func newAddIndexWorker(sessCtx sessionctx.Context, worker *worker, id int, t table.Table, indexInfo *model.IndexInfo, colFieldMap map[int64]*types.FieldType) *addIndexWorker { - index := tables.NewIndex(t.GetID(), t.Meta(), indexInfo) +func newAddIndexWorker(sessCtx sessionctx.Context, worker *worker, id int, t table.PhysicalTable, indexInfo *model.IndexInfo, colFieldMap map[int64]*types.FieldType) *addIndexWorker { + index := tables.NewIndex(t.GetPhysicalID(), t.Meta(), indexInfo) return &addIndexWorker{ id: id, ddlWorker: worker, @@ -804,7 +804,7 @@ func (w *addIndexWorker) handleBackfillTask(d *ddlCtx, task *reorgIndexTask) *ad rightParenthesis = "]" } log.Infof("[ddl-reorg] worker(%v), finish region %v ranges [%v,%v%s, addedCount:%v, scanCount:%v, nextHandle:%v, elapsed time(s):%v", - w.id, task.partitionID, task.startHandle, task.endHandle, rightParenthesis, result.addedCount, result.scanCount, result.nextHandle, time.Since(startTime).Seconds()) + w.id, task.physicalTableID, task.startHandle, task.endHandle, rightParenthesis, result.addedCount, result.scanCount, result.nextHandle, time.Since(startTime).Seconds()) return result } @@ -856,11 +856,11 @@ func makeupIndexColFieldMap(t table.Table, indexInfo *model.IndexInfo) map[int64 // splitTableRanges uses PD region's key ranges to split the backfilling table key range space, // to speed up adding index in table with disperse handle. // The `t` should be a non-partitioned table or a partition. -func splitTableRanges(t table.Table, store kv.Storage, startHandle, endHandle int64) ([]kv.KeyRange, error) { +func splitTableRanges(t table.PhysicalTable, store kv.Storage, startHandle, endHandle int64) ([]kv.KeyRange, error) { startRecordKey := t.RecordKey(startHandle) endRecordKey := t.RecordKey(endHandle).Next() - log.Infof("[ddl-reorg] split partition %v range [%v, %v] from PD", t.GetID(), startHandle, endHandle) + log.Infof("[ddl-reorg] split partition %v range [%v, %v] from PD", t.GetPhysicalID(), startHandle, endHandle) kvRange := kv.KeyRange{StartKey: startRecordKey, EndKey: endRecordKey} s, ok := store.(tikv.Storage) if !ok { @@ -947,7 +947,7 @@ func (w *worker) handleReorgTasks(reorgInfo *reorgInfo, totalAddedCount *int64, if err != nil { // update the reorg handle that has been processed. err1 := kv.RunInNewTxn(reorgInfo.d.store, true, func(txn kv.Transaction) error { - return errors.Trace(reorgInfo.UpdateReorgMeta(txn, nextHandle, reorgInfo.EndHandle, reorgInfo.PartitionID)) + return errors.Trace(reorgInfo.UpdateReorgMeta(txn, nextHandle, reorgInfo.EndHandle, reorgInfo.PhysicalTableID)) }) metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime) log.Warnf("[ddl-reorg] total added index for %d rows, this task [%d,%d) add index for %d failed %v, take time %v, update handle err %v", @@ -966,7 +966,7 @@ func (w *worker) handleReorgTasks(reorgInfo *reorgInfo, totalAddedCount *int64, // sendRangeTaskToWorkers sends tasks to workers, and returns remaining kvRanges that is not handled. func (w *worker) sendRangeTaskToWorkers(t table.Table, workers []*addIndexWorker, reorgInfo *reorgInfo, totalAddedCount *int64, kvRanges []kv.KeyRange) ([]kv.KeyRange, error) { batchTasks := make([]*reorgIndexTask, 0, len(workers)) - partitionID := reorgInfo.PartitionID + physicalTableID := reorgInfo.PhysicalTableID // Build reorg indices tasks. for _, keyRange := range kvRanges { @@ -980,7 +980,7 @@ func (w *worker) sendRangeTaskToWorkers(t table.Table, workers []*addIndexWorker if endKey.Cmp(keyRange.EndKey) < 0 { endIncluded = true } - task := &reorgIndexTask{partitionID, startHandle, endHandle, endIncluded} + task := &reorgIndexTask{physicalTableID, startHandle, endHandle, endIncluded} batchTasks = append(batchTasks, task) if len(batchTasks) >= len(workers) { @@ -1009,7 +1009,7 @@ func (w *worker) sendRangeTaskToWorkers(t table.Table, workers []*addIndexWorker // buildIndexForReorgInfo build backfilling tasks from [reorgInfo.StartHandle, reorgInfo.EndHandle), // and send these tasks to add index workers, till we finish adding the indices. -func (w *worker) buildIndexForReorgInfo(t table.Table, workers []*addIndexWorker, job *model.Job, reorgInfo *reorgInfo) error { +func (w *worker) buildIndexForReorgInfo(t table.PhysicalTable, workers []*addIndexWorker, job *model.Job, reorgInfo *reorgInfo) error { totalAddedCount := job.GetRowCount() startHandle, endHandle := reorgInfo.StartHandle, reorgInfo.EndHandle @@ -1050,7 +1050,7 @@ func (w *worker) buildIndexForReorgInfo(t table.Table, workers []*addIndexWorker // 4. Wait all these running tasks finished, then continue to step 3, until all tasks is done. // The above operations are completed in a transaction. // Finally, update the concurrent processing of the total number of rows, and store the completed handle value. -func (w *worker) addPhysicalTableIndex(t table.Table, indexInfo *model.IndexInfo, reorgInfo *reorgInfo) error { +func (w *worker) addPhysicalTableIndex(t table.PhysicalTable, indexInfo *model.IndexInfo, reorgInfo *reorgInfo) error { job := reorgInfo.Job log.Infof("[ddl-reorg] addTableIndex, job:%s, reorgInfo:%#v", job, reorgInfo) colFieldMap := makeupIndexColFieldMap(t, indexInfo) @@ -1075,9 +1075,9 @@ func (w *worker) addTableIndex(t table.Table, idx *model.IndexInfo, reorgInfo *r if tbl, ok := t.(table.PartitionedTable); ok { var finish bool for !finish { - p := tbl.GetPartition(reorgInfo.PartitionID) + p := tbl.GetPartition(reorgInfo.PhysicalTableID) if p == nil { - return errors.Errorf("Can not find partition id %d for table %d", reorgInfo.PartitionID, t.Meta().ID) + return errors.Errorf("Can not find partition id %d for table %d", reorgInfo.PhysicalTableID, t.Meta().ID) } err = w.addPhysicalTableIndex(p, idx, reorgInfo) if err != nil { @@ -1089,7 +1089,7 @@ func (w *worker) addTableIndex(t table.Table, idx *model.IndexInfo, reorgInfo *r } } } else { - err = w.addPhysicalTableIndex(t, idx, reorgInfo) + err = w.addPhysicalTableIndex(t.(table.PhysicalTable), idx, reorgInfo) } return errors.Trace(err) } @@ -1103,7 +1103,7 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo return true, nil } - pid, err := findNextPartitionID(reorg.PartitionID, pi.Definitions) + pid, err := findNextPartitionID(reorg.PhysicalTableID, pi.Definitions) if err != nil { // Fatal error, should not run here. log.Errorf("[ddl-reorg] update reorg fail, %v error stack: %s", t, errors.ErrorStack(err)) @@ -1119,27 +1119,27 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo return false, errors.Trace(err) } log.Infof("[ddl-reorg] job %v update reorgInfo partition %d range [%d %d]", reorg.Job.ID, pid, start, end) - reorg.StartHandle, reorg.EndHandle, reorg.PartitionID = start, end, pid + reorg.StartHandle, reorg.EndHandle, reorg.PhysicalTableID = start, end, pid // Write the reorg info to store so the whole reorganize process can recover from panic. err = kv.RunInNewTxn(reorg.d.store, true, func(txn kv.Transaction) error { - return errors.Trace(reorg.UpdateReorgMeta(txn, reorg.StartHandle, reorg.EndHandle, reorg.PartitionID)) + return errors.Trace(reorg.UpdateReorgMeta(txn, reorg.StartHandle, reorg.EndHandle, reorg.PhysicalTableID)) }) return false, errors.Trace(err) } // findNextPartitionID finds the next partition ID in the PartitionDefinition array. -// Returns 0 if current partitionID is already the last one. -func findNextPartitionID(partitionID int64, defs []model.PartitionDefinition) (int64, error) { +// Returns 0 if current partition is already the last one. +func findNextPartitionID(currentPartition int64, defs []model.PartitionDefinition) (int64, error) { for i, def := range defs { - if partitionID == def.ID { + if currentPartition == def.ID { if i == len(defs)-1 { return 0, nil } return defs[i+1].ID, nil } } - return 0, errors.Errorf("partition id not found %d", partitionID) + return 0, errors.Errorf("partition id not found %d", currentPartition) } func findIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInfo { diff --git a/ddl/partition.go b/ddl/partition.go index c9dd67a192658..c8295dfc23753 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -251,7 +251,7 @@ func onDropTablePartition(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - partitionID := removePartitionInfo(tblInfo, partName) + physicalTableID := removePartitionInfo(tblInfo, partName) ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) @@ -260,7 +260,7 @@ func onDropTablePartition(t *meta.Meta, job *model.Job) (ver int64, _ error) { // Finish this job. job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) // A background job will be created to delete old partition data. - job.Args = []interface{}{partitionID} + job.Args = []interface{}{physicalTableID} return ver, nil } @@ -275,11 +275,11 @@ func getPartitionIDs(table *model.TableInfo) []int64 { if table.GetPartitionInfo() == nil { return []int64{} } - partitionIDs := make([]int64, 0, len(table.Partition.Definitions)) + physicalTableIDs := make([]int64, 0, len(table.Partition.Definitions)) for _, def := range table.Partition.Definitions { - partitionIDs = append(partitionIDs, def.ID) + physicalTableIDs = append(physicalTableIDs, def.ID) } - return partitionIDs + return physicalTableIDs } // checkRangePartitioningKeysConstraints checks that the range partitioning key is included in the table constraint. diff --git a/ddl/reorg.go b/ddl/reorg.go index 3f854c028a6a8..bb9170cc34c82 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -184,16 +184,16 @@ type reorgInfo struct { EndHandle int64 d *ddlCtx first bool - // PartitionID is used for partitioned table. + // PhysicalTableID is used for partitioned table. // DDL reorganize for a partitioned table will handle partitions one by one, - // PartitionID is used to trace the current partition we are handling. - // If the table is not partitioned, PartitionID would be TableID. - PartitionID int64 + // PhysicalTableID is used to trace the current partition we are handling. + // If the table is not partitioned, PhysicalTableID would be TableID. + PhysicalTableID int64 } -func constructDescTableScanPB(partitionID int64, pbColumnInfos []*tipb.ColumnInfo) *tipb.Executor { +func constructDescTableScanPB(physicalTableID int64, pbColumnInfos []*tipb.ColumnInfo) *tipb.Executor { tblScan := &tipb.TableScan{ - TableId: partitionID, + TableId: physicalTableID, Columns: pbColumnInfos, Desc: true, } @@ -208,7 +208,7 @@ func constructLimitPB(count uint64) *tipb.Executor { return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec} } -func buildDescTableScanDAG(startTS uint64, tbl table.Table, columns []*model.ColumnInfo, limit uint64) (*tipb.DAGRequest, error) { +func buildDescTableScanDAG(startTS uint64, tbl table.PhysicalTable, columns []*model.ColumnInfo, limit uint64) (*tipb.DAGRequest, error) { dagReq := &tipb.DAGRequest{} dagReq.StartTs = startTS _, timeZoneOffset := time.Now().In(time.UTC).Zone() @@ -219,7 +219,7 @@ func buildDescTableScanDAG(startTS uint64, tbl table.Table, columns []*model.Col dagReq.Flags |= model.FlagInSelectStmt pbColumnInfos := model.ColumnsToProto(columns, tbl.Meta().PKIsHandle) - tblScanExec := constructDescTableScanPB(tbl.GetID(), pbColumnInfos) + tblScanExec := constructDescTableScanPB(tbl.GetPhysicalID(), pbColumnInfos) dagReq.Executors = append(dagReq.Executors, tblScanExec) dagReq.Executors = append(dagReq.Executors, constructLimitPB(limit)) return dagReq, nil @@ -234,14 +234,14 @@ func getColumnsTypes(columns []*model.ColumnInfo) []*types.FieldType { } // buildDescTableScan builds a desc table scan upon tblInfo. -func (d *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl table.Table, columns []*model.ColumnInfo, limit uint64) (distsql.SelectResult, error) { +func (d *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl table.PhysicalTable, columns []*model.ColumnInfo, limit uint64) (distsql.SelectResult, error) { dagPB, err := buildDescTableScanDAG(startTS, tbl, columns, limit) if err != nil { return nil, errors.Trace(err) } ranges := ranger.FullIntRange(false) var builder distsql.RequestBuilder - builder.SetTableRanges(tbl.GetID(), ranges, nil). + builder.SetTableRanges(tbl.GetPhysicalID(), ranges, nil). SetDAGRequest(dagPB). SetKeepOrder(true). SetConcurrency(1).SetDesc(true) @@ -260,7 +260,7 @@ func (d *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl tab } // GetTableMaxRowID gets the last row id of the table partition. -func (d *ddlCtx) GetTableMaxRowID(startTS uint64, tbl table.Table) (maxRowID int64, emptyTable bool, err error) { +func (d *ddlCtx) GetTableMaxRowID(startTS uint64, tbl table.PhysicalTable) (maxRowID int64, emptyTable bool, err error) { maxRowID = int64(math.MaxInt64) var columns []*model.ColumnInfo if tbl.Meta().PKIsHandle { @@ -300,7 +300,7 @@ func (d *ddlCtx) GetTableMaxRowID(startTS uint64, tbl table.Table) (maxRowID int var gofailOnceGuard bool // getTableRange gets the start and end handle of a table (or partition). -func getTableRange(d *ddlCtx, tbl table.Table, snapshotVer uint64, priority int) (startHandle, endHandle int64, err error) { +func getTableRange(d *ddlCtx, tbl table.PhysicalTable, snapshotVer uint64, priority int) (startHandle, endHandle int64, err error) { startHandle = math.MinInt64 endHandle = math.MaxInt64 // Get the start handle of this partition. @@ -319,7 +319,7 @@ func getTableRange(d *ddlCtx, tbl table.Table, snapshotVer uint64, priority int) return 0, 0, errors.Trace(err) } if endHandle < startHandle || emptyTable { - log.Infof("[ddl-reorg] get table range %v endHandle < startHandle partition %d [%d %d]", tbl.Meta(), tbl.GetID(), endHandle, startHandle) + log.Infof("[ddl-reorg] get table range %v endHandle < startHandle partition %d [%d %d]", tbl.Meta(), tbl.GetPhysicalID(), endHandle, startHandle) endHandle = startHandle } return @@ -346,12 +346,14 @@ func getReorgInfo(d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table) (*re } tblInfo := tbl.Meta() pid = tblInfo.ID - tp := tbl + var tb table.PhysicalTable if pi := tblInfo.GetPartitionInfo(); pi != nil { pid = pi.Definitions[0].ID - tp = tbl.(table.PartitionedTable).GetPartition(pid) + tb = tbl.(table.PartitionedTable).GetPartition(pid) + } else { + tb = tbl.(table.PhysicalTable) } - start, end, err = getTableRange(d, tp, ver.Ver, job.Priority) + start, end, err = getTableRange(d, tb, ver.Ver, job.Priority) if err != nil { return nil, errors.Trace(err) } @@ -379,12 +381,12 @@ func getReorgInfo(d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table) (*re info.d = d info.StartHandle = start info.EndHandle = end - info.PartitionID = pid + info.PhysicalTableID = pid return &info, errors.Trace(err) } -func (r *reorgInfo) UpdateReorgMeta(txn kv.Transaction, startHandle, endHandle, partitionID int64) error { +func (r *reorgInfo) UpdateReorgMeta(txn kv.Transaction, startHandle, endHandle, physicalTableID int64) error { t := meta.NewMeta(txn) - return errors.Trace(t.UpdateDDLReorgHandle(r.Job, startHandle, endHandle, partitionID)) + return errors.Trace(t.UpdateDDLReorgHandle(r.Job, startHandle, endHandle, physicalTableID)) } diff --git a/executor/analyze.go b/executor/analyze.go index 0c172d38226a0..853a004c421c2 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -79,7 +79,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, chk *chunk.Chunk) error { continue } for i, hg := range result.Hist { - err1 := statsHandle.SaveStatsToStorage(result.PhysicalID, result.Count, result.IsIndex, hg, result.Cms[i], 1) + err1 := statsHandle.SaveStatsToStorage(result.PhysicalTableID, result.Count, result.IsIndex, hg, result.Cms[i], 1) if err1 != nil { err = err1 log.Error(errors.ErrorStack(err)) @@ -147,10 +147,10 @@ func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) statistics.AnalyzeResult { return statistics.AnalyzeResult{Err: err} } result := statistics.AnalyzeResult{ - PhysicalID: idxExec.physicalID, - Hist: []*statistics.Histogram{hist}, - Cms: []*statistics.CMSketch{cms}, - IsIndex: 1, + PhysicalTableID: idxExec.physicalTableID, + Hist: []*statistics.Histogram{hist}, + Cms: []*statistics.CMSketch{cms}, + IsIndex: 1, } if hist.Len() > 0 { result.Count = hist.Buckets[hist.Len()-1].Count @@ -160,18 +160,18 @@ func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) statistics.AnalyzeResult { // AnalyzeIndexExec represents analyze index push down executor. type AnalyzeIndexExec struct { - ctx sessionctx.Context - physicalID int64 // physicalID is the partition id for a partitioned table, otherwise, it is the table id. - idxInfo *model.IndexInfo - concurrency int - priority int - analyzePB *tipb.AnalyzeReq - result distsql.SelectResult + ctx sessionctx.Context + physicalTableID int64 + idxInfo *model.IndexInfo + concurrency int + priority int + analyzePB *tipb.AnalyzeReq + result distsql.SelectResult } func (e *AnalyzeIndexExec) open() error { var builder distsql.RequestBuilder - kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalID, e.idxInfo.ID, ranger.FullRange()). + kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.idxInfo.ID, ranger.FullRange()). SetAnalyzeRequest(e.analyzePB). SetKeepOrder(true). Build() @@ -232,9 +232,9 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) statistics.AnalyzeResul return statistics.AnalyzeResult{Err: err} } result := statistics.AnalyzeResult{ - PhysicalID: colExec.physicalID, - Hist: hists, - Cms: cms, + PhysicalTableID: colExec.physicalTableID, + Hist: hists, + Cms: cms, } hist := hists[0] result.Count = hist.NullCount @@ -246,15 +246,15 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) statistics.AnalyzeResul // AnalyzeColumnsExec represents Analyze columns push down executor. type AnalyzeColumnsExec struct { - ctx sessionctx.Context - physicalID int64 // physicalID is the partition id for a partitioned table, otherwise, it is the table id. - colsInfo []*model.ColumnInfo - pkInfo *model.ColumnInfo - concurrency int - priority int - keepOrder bool - analyzePB *tipb.AnalyzeReq - resultHandler *tableResultHandler + ctx sessionctx.Context + physicalTableID int64 + colsInfo []*model.ColumnInfo + pkInfo *model.ColumnInfo + concurrency int + priority int + keepOrder bool + analyzePB *tipb.AnalyzeReq + resultHandler *tableResultHandler } func (e *AnalyzeColumnsExec) open() error { @@ -286,7 +286,7 @@ func (e *AnalyzeColumnsExec) open() error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder - kvReq, err := builder.SetTableRanges(e.physicalID, ranges, nil). + kvReq, err := builder.SetTableRanges(e.physicalTableID, ranges, nil). SetAnalyzeRequest(e.analyzePB). SetKeepOrder(e.keepOrder). Build() diff --git a/executor/builder.go b/executor/builder.go index 7122e5f7a33a1..c5761c6b98932 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1303,10 +1303,10 @@ func (b *executorBuilder) buildDelete(v *plan.Delete) Executor { func (b *executorBuilder) buildAnalyzeIndexPushdown(task plan.AnalyzeIndexTask) *AnalyzeIndexExec { _, offset := zone(b.ctx) e := &AnalyzeIndexExec{ - ctx: b.ctx, - physicalID: task.PhysicalID, - idxInfo: task.IndexInfo, - concurrency: b.ctx.GetSessionVars().IndexSerialScanConcurrency, + ctx: b.ctx, + physicalTableID: task.PhysicalTableID, + idxInfo: task.IndexInfo, + concurrency: b.ctx.GetSessionVars().IndexSerialScanConcurrency, analyzePB: &tipb.AnalyzeReq{ Tp: tipb.AnalyzeType_TypeIndex, StartTs: math.MaxUint64, @@ -1335,12 +1335,12 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plan.AnalyzeColumnsTa _, offset := zone(b.ctx) e := &AnalyzeColumnsExec{ - ctx: b.ctx, - physicalID: task.PhysicalID, - colsInfo: task.ColsInfo, - pkInfo: task.PKInfo, - concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency, - keepOrder: keepOrder, + ctx: b.ctx, + physicalTableID: task.PhysicalTableID, + colsInfo: task.ColsInfo, + pkInfo: task.PKInfo, + concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency, + keepOrder: keepOrder, analyzePB: &tipb.AnalyzeReq{ Tp: tipb.AnalyzeType_TypeColumn, StartTs: math.MaxUint64, @@ -1534,20 +1534,20 @@ func buildNoRangeTableReader(b *executorBuilder, v *plan.PhysicalTableReader) (* ts := v.TablePlans[0].(*plan.PhysicalTableScan) table, _ := b.is.TableByID(ts.Table.ID) e := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), - dagPB: dagReq, - tableID: ts.Table.ID, - table: table, - keepOrder: ts.KeepOrder, - desc: ts.Desc, - columns: ts.Columns, - streaming: streaming, - corColInFilter: b.corColInDistPlan(v.TablePlans), - corColInAccess: b.corColInAccess(v.TablePlans[0]), - plans: v.TablePlans, - } - if isPartition, partitionID := ts.IsPartition(); isPartition { - e.tableID = partitionID + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + dagPB: dagReq, + physicalTableID: ts.Table.ID, + table: table, + keepOrder: ts.KeepOrder, + desc: ts.Desc, + columns: ts.Columns, + streaming: streaming, + corColInFilter: b.corColInDistPlan(v.TablePlans), + corColInAccess: b.corColInAccess(v.TablePlans[0]), + plans: v.TablePlans, + } + if isPartition, physicalTableID := ts.IsPartition(); isPartition { + e.physicalTableID = physicalTableID } if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) @@ -1588,23 +1588,23 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plan.PhysicalIndexReader) (* is := v.IndexPlans[0].(*plan.PhysicalIndexScan) table, _ := b.is.TableByID(is.Table.ID) e := &IndexReaderExecutor{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), - dagPB: dagReq, - tableID: is.Table.ID, - table: table, - index: is.Index, - keepOrder: is.KeepOrder, - desc: is.Desc, - columns: is.Columns, - streaming: streaming, - corColInFilter: b.corColInDistPlan(v.IndexPlans), - corColInAccess: b.corColInAccess(v.IndexPlans[0]), - idxCols: is.IdxCols, - colLens: is.IdxColLens, - plans: v.IndexPlans, - } - if isPartition, partitionID := is.IsPartition(); isPartition { - e.tableID = partitionID + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + dagPB: dagReq, + physicalTableID: is.Table.ID, + table: table, + index: is.Index, + keepOrder: is.KeepOrder, + desc: is.Desc, + columns: is.Columns, + streaming: streaming, + corColInFilter: b.corColInDistPlan(v.IndexPlans), + corColInAccess: b.corColInAccess(v.IndexPlans[0]), + idxCols: is.IdxCols, + colLens: is.IdxColLens, + plans: v.IndexPlans, + } + if isPartition, physicalTableID := is.IsPartition(); isPartition { + e.physicalTableID = physicalTableID } if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) @@ -1657,7 +1657,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plan.PhysicalIndexLook e := &IndexLookUpExecutor{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), dagPB: indexReq, - tableID: is.Table.ID, + physicalTableID: is.Table.ID, table: table, index: is.Index, keepOrder: is.KeepOrder, @@ -1675,8 +1675,8 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plan.PhysicalIndexLook idxPlans: v.IndexPlans, tblPlans: v.TablePlans, } - if isPartition, partitionID := ts.IsPartition(); isPartition { - e.tableID = partitionID + if isPartition, physicalTableID := ts.IsPartition(); isPartition { + e.physicalTableID = physicalTableID } if containsLimit(indexReq.Executors) { @@ -1751,7 +1751,7 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Context, e *TableReaderExecutor, handles []int64) (Executor, error) { sort.Sort(sortutil.Int64Slice(handles)) var b distsql.RequestBuilder - kvReq, err := b.SetTableHandles(e.tableID, handles). + kvReq, err := b.SetTableHandles(e.physicalTableID, handles). SetDAGRequest(e.dagPB). SetDesc(e.desc). SetKeepOrder(e.keepOrder). @@ -1777,7 +1777,7 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte if err != nil { return nil, errors.Trace(err) } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) if err != nil { return nil, errors.Trace(err) } @@ -1791,7 +1791,7 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context if err != nil { return nil, errors.Trace(err) } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/distsql.go b/executor/distsql.go index 373a5a81ca8a4..2cb5ca8e4b5da 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -241,13 +241,13 @@ func rebuildIndexRanges(ctx sessionctx.Context, is *plan.PhysicalIndexScan, idxC type IndexReaderExecutor struct { baseExecutor - table table.Table - index *model.IndexInfo - tableID int64 - keepOrder bool - desc bool - ranges []*ranger.Range - dagPB *tipb.DAGRequest + table table.Table + index *model.IndexInfo + physicalTableID int64 + keepOrder bool + desc bool + ranges []*ranger.Range + dagPB *tipb.DAGRequest // result returns one or more distsql.PartialResult and each PartialResult is returned by one region. result distsql.SelectResult @@ -289,7 +289,7 @@ func (e *IndexReaderExecutor) Open(ctx context.Context) error { return errors.Trace(err) } } - kvRanges, err := distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, e.ranges, e.feedback) + kvRanges, err := distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, e.ranges, e.feedback) if err != nil { e.feedback.Invalidate() return errors.Trace(err) @@ -334,13 +334,13 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) type IndexLookUpExecutor struct { baseExecutor - table table.Table - index *model.IndexInfo - tableID int64 - keepOrder bool - desc bool - ranges []*ranger.Range - dagPB *tipb.DAGRequest + table table.Table + index *model.IndexInfo + physicalTableID int64 + keepOrder bool + desc bool + ranges []*ranger.Range + dagPB *tipb.DAGRequest // handleIdx is the index of handle, which is only used for case of keeping order. handleIdx int tableRequest *tipb.DAGRequest @@ -383,7 +383,7 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { return errors.Trace(err) } } - kvRanges, err := distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.tableID, e.index.ID, e.ranges, e.feedback) + kvRanges, err := distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, e.ranges, e.feedback) if err != nil { e.feedback.Invalidate() return errors.Trace(err) @@ -511,14 +511,14 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.id+"_tableReader"), - table: e.table, - tableID: e.tableID, - dagPB: e.tableRequest, - streaming: e.tableStreaming, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), - corColInFilter: e.corColInTblSide, - plans: e.tblPlans, + baseExecutor: newBaseExecutor(e.ctx, e.schema, e.id+"_tableReader"), + table: e.table, + physicalTableID: e.physicalTableID, + dagPB: e.tableRequest, + streaming: e.tableStreaming, + feedback: statistics.NewQueryFeedback(0, nil, 0, false), + corColInFilter: e.corColInTblSide, + plans: e.tblPlans, }, handles) if err != nil { log.Error(err) diff --git a/executor/table_reader.go b/executor/table_reader.go index a04b5897213c8..a2e7abe88b446 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -33,12 +33,12 @@ var _ Executor = &TableReaderExecutor{} type TableReaderExecutor struct { baseExecutor - table table.Table - tableID int64 - keepOrder bool - desc bool - ranges []*ranger.Range - dagPB *tipb.DAGRequest + table table.Table + physicalTableID int64 + keepOrder bool + desc bool + ranges []*ranger.Range + dagPB *tipb.DAGRequest // columns are only required by union scan. columns []*model.ColumnInfo @@ -119,7 +119,7 @@ func (e *TableReaderExecutor) Close() error { // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder - kvReq, err := builder.SetTableRanges(e.tableID, ranges, e.feedback). + kvReq, err := builder.SetTableRanges(e.physicalTableID, ranges, e.feedback). SetDAGRequest(e.dagPB). SetDesc(e.desc). SetKeepOrder(e.keepOrder). diff --git a/infoschema/tables.go b/infoschema/tables.go index 02346da318ec8..3e5507a156c01 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1343,7 +1343,7 @@ func (it *infoschemaTable) Meta() *model.TableInfo { return it.meta } -func (it *infoschemaTable) GetID() int64 { +func (it *infoschemaTable) GetPhysicalID() int64 { return it.meta.ID } diff --git a/meta/meta.go b/meta/meta.go index e6c3a23fcd016..443adda7a9161 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -618,7 +618,7 @@ func (m *Meta) reorgJobEndHandle(id int64) []byte { return b } -func (m *Meta) reorgJobPartitionID(id int64) []byte { +func (m *Meta) reorgJobPhysicalTableID(id int64) []byte { b := make([]byte, 8, 12) binary.BigEndian.PutUint64(b, uint64(id)) b = append(b, "_pid"...) @@ -715,7 +715,7 @@ func (m *Meta) UpdateDDLReorgStartHandle(job *model.Job, startHandle int64) erro } // UpdateDDLReorgHandle saves the job reorganization latest processed information for later resuming. -func (m *Meta) UpdateDDLReorgHandle(job *model.Job, startHandle, endHandle, partitionID int64) error { +func (m *Meta) UpdateDDLReorgHandle(job *model.Job, startHandle, endHandle, physicalTableID int64) error { err := m.txn.HSet(mDDLJobReorgKey, m.reorgJobStartHandle(job.ID), []byte(strconv.FormatInt(startHandle, 10))) if err != nil { return errors.Trace(err) @@ -724,7 +724,7 @@ func (m *Meta) UpdateDDLReorgHandle(job *model.Job, startHandle, endHandle, part if err != nil { return errors.Trace(err) } - err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobPartitionID(job.ID), []byte(strconv.FormatInt(partitionID, 10))) + err = m.txn.HSet(mDDLJobReorgKey, m.reorgJobPhysicalTableID(job.ID), []byte(strconv.FormatInt(physicalTableID, 10))) return errors.Trace(err) } @@ -737,14 +737,14 @@ func (m *Meta) RemoveDDLReorgHandle(job *model.Job) error { if err = m.txn.HDel(mDDLJobReorgKey, m.reorgJobEndHandle(job.ID)); err != nil { log.Warn("remove ddl reorg end handle error:", err) } - if err = m.txn.HDel(mDDLJobReorgKey, m.reorgJobPartitionID(job.ID)); err != nil { - log.Warn("remove ddl reorg partition id error:", err) + if err = m.txn.HDel(mDDLJobReorgKey, m.reorgJobPhysicalTableID(job.ID)); err != nil { + log.Warn("remove ddl reorg physical id error:", err) } return nil } // GetDDLReorgHandle gets the latest processed DDL reorganize position. -func (m *Meta) GetDDLReorgHandle(job *model.Job) (startHandle, endHandle, partitionID int64, err error) { +func (m *Meta) GetDDLReorgHandle(job *model.Job) (startHandle, endHandle, physicalTableID int64, err error) { startHandle, err = m.txn.HGetInt64(mDDLJobReorgKey, m.reorgJobStartHandle(job.ID)) if err != nil { err = errors.Trace(err) @@ -755,21 +755,21 @@ func (m *Meta) GetDDLReorgHandle(job *model.Job) (startHandle, endHandle, partit err = errors.Trace(err) return } - partitionID, err = m.txn.HGetInt64(mDDLJobReorgKey, m.reorgJobPartitionID(job.ID)) + physicalTableID, err = m.txn.HGetInt64(mDDLJobReorgKey, m.reorgJobPhysicalTableID(job.ID)) if err != nil { err = errors.Trace(err) return } - // endHandle or partitionID may be 0, because older version TiDB (without table partition) doesn't store them. + // endHandle or physicalTableID may be 0, because older version TiDB (without table partition) doesn't store them. // update them to table's in this case. - if endHandle == 0 || partitionID == 0 { + if endHandle == 0 || physicalTableID == 0 { if job.ReorgMeta != nil { endHandle = job.ReorgMeta.EndHandle } else { endHandle = math.MaxInt64 } - partitionID = job.TableID - log.Warnf("new TiDB binary running on old TiDB ddl reorg data, partition %v [%v %v]", partitionID, startHandle, endHandle) + physicalTableID = job.TableID + log.Warnf("new TiDB binary running on old TiDB ddl reorg data, partition %v [%v %v]", physicalTableID, startHandle, endHandle) } return } diff --git a/perfschema/tables.go b/perfschema/tables.go index 89bc3ae2cbc7a..8f89614152cf4 100644 --- a/perfschema/tables.go +++ b/perfschema/tables.go @@ -142,7 +142,7 @@ func (vt *perfSchemaTable) Meta() *model.TableInfo { } // GetID implements table.Table GetID interface. -func (vt *perfSchemaTable) GetID() int64 { +func (vt *perfSchemaTable) GetPhysicalID() int64 { return vt.meta.ID } diff --git a/plan/common_plans.go b/plan/common_plans.go index d5d2c82e7cebf..15a351197bfb8 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -355,15 +355,16 @@ type Delete struct { // AnalyzeColumnsTask is used for analyze columns. type AnalyzeColumnsTask struct { - PhysicalID int64 // PhysicalID is the partition id for a partitioned table, otherwise, it is the table id. - PKInfo *model.ColumnInfo - ColsInfo []*model.ColumnInfo + PhysicalTableID int64 + PKInfo *model.ColumnInfo + ColsInfo []*model.ColumnInfo } // AnalyzeIndexTask is used for analyze index. type AnalyzeIndexTask struct { - PhysicalID int64 // PhysicalID is the partition id for a partitioned table, otherwise, it is the table id. - IndexInfo *model.IndexInfo + // PhysicalTableID is the id for a partition or a table. + PhysicalTableID int64 + IndexInfo *model.IndexInfo } // Analyze represents an analyze plan diff --git a/plan/find_best_task.go b/plan/find_best_task.go index 87e8e8e46589a..58c4e2d37a009 100644 --- a/plan/find_best_task.go +++ b/plan/find_best_task.go @@ -330,7 +330,7 @@ func (ds *DataSource) convertToIndexScan(prop *requiredProp, path *accessPath) ( filterCondition: path.indexFilters, dataSourceSchema: ds.schema, isPartition: ds.isPartition, - partitionID: ds.partitionID, + physicalTableID: ds.physicalTableID, }.init(ds.ctx) statsTbl := ds.statisticTable if statsTbl.Indices[idx.ID] != nil { @@ -345,10 +345,10 @@ func (ds *DataSource) convertToIndexScan(prop *requiredProp, path *accessPath) ( } // On this way, it's double read case. ts := PhysicalTableScan{ - Columns: ds.Columns, - Table: is.Table, - isPartition: ds.isPartition, - partitionID: ds.partitionID, + Columns: ds.Columns, + Table: is.Table, + isPartition: ds.isPartition, + physicalTableID: ds.physicalTableID, }.init(ds.ctx) ts.SetSchema(ds.schema.Clone()) cop.tablePlan = ts @@ -522,12 +522,12 @@ func (ds *DataSource) convertToTableScan(prop *requiredProp, path *accessPath) ( } ts := PhysicalTableScan{ - Table: ds.tableInfo, - Columns: ds.Columns, - TableAsName: ds.TableAsName, - DBName: ds.DBName, - isPartition: ds.isPartition, - partitionID: ds.partitionID, + Table: ds.tableInfo, + Columns: ds.Columns, + TableAsName: ds.TableAsName, + DBName: ds.DBName, + isPartition: ds.isPartition, + physicalTableID: ds.physicalTableID, }.init(ds.ctx) ts.SetSchema(ds.schema) var pkCol *expression.Column diff --git a/plan/logical_plans.go b/plan/logical_plans.go index fd17388851c41..1e4ad4a859292 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -301,8 +301,8 @@ type DataSource struct { possibleAccessPaths []*accessPath // The data source may be a partition, rather than a real table. - isPartition bool - partitionID int64 + isPartition bool + physicalTableID int64 } // accessPath tells how we access one index or just access table. diff --git a/plan/physical_plans.go b/plan/physical_plans.go index 29df0d59de3eb..9d629f189dcc3 100644 --- a/plan/physical_plans.go +++ b/plan/physical_plans.go @@ -116,8 +116,8 @@ type PhysicalIndexScan struct { rangeDecidedBy []*expression.Column // The index scan may be on a partition. - isPartition bool - partitionID int64 + isPartition bool + physicalTableID int64 } // PhysicalMemTable reads memory table. @@ -155,15 +155,15 @@ type PhysicalTableScan struct { Hist *statistics.Histogram // The table scan may be a partition, rather than a real table. - isPartition bool - partitionID int64 + isPartition bool + physicalTableID int64 rangeDecidedBy []*expression.Column } // IsPartition returns true and partition ID if it's actually a partition. func (ts *PhysicalTableScan) IsPartition() (bool, int64) { - return ts.isPartition, ts.partitionID + return ts.isPartition, ts.physicalTableID } // PhysicalProjection is the physical operator of projection. @@ -343,7 +343,7 @@ type PhysicalUnionScan struct { // IsPartition returns true and partition ID if it works on a partition. func (p *PhysicalIndexScan) IsPartition() (bool, int64) { - return p.isPartition, p.partitionID + return p.isPartition, p.physicalTableID } // IsPointGetByUniqueKey checks whether is a point get by unique key. diff --git a/plan/planbuilder.go b/plan/planbuilder.go index a24fcfe6bedb4..bdacf33edea5f 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -590,12 +590,12 @@ func (b *planBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) Plan { physicalIDs := getPhysicalIDs(tbl.TableInfo) for _, idx := range idxInfo { for _, id := range physicalIDs { - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalID: id, IndexInfo: idx}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalTableID: id, IndexInfo: idx}) } } if len(colInfo) > 0 || pkInfo != nil { for _, id := range physicalIDs { - p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PhysicalID: id, PKInfo: pkInfo, ColsInfo: colInfo}) + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PhysicalTableID: id, PKInfo: pkInfo, ColsInfo: colInfo}) } } } @@ -612,7 +612,7 @@ func (b *planBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) return nil, ErrAnalyzeMissIndex.GenByArgs(idxName.O, tblInfo.Name.O) } for _, id := range physicalIDs { - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalID: id, IndexInfo: idx}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalTableID: id, IndexInfo: idx}) } } return p, nil @@ -625,7 +625,7 @@ func (b *planBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) Plan { for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for _, id := range physicalIDs { - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalID: id, IndexInfo: idx}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{PhysicalTableID: id, IndexInfo: idx}) } } } diff --git a/plan/rule_partition_processor.go b/plan/rule_partition_processor.go index 0276d16a427f0..3015ee0785400 100644 --- a/plan/rule_partition_processor.go +++ b/plan/rule_partition_processor.go @@ -104,7 +104,7 @@ func (s *partitionProcessor) prune(ds *DataSource) (LogicalPlan, error) { newDataSource := *ds newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.context(), TypeTableScan, &newDataSource) newDataSource.isPartition = true - newDataSource.partitionID = pi.Definitions[i].ID + newDataSource.physicalTableID = pi.Definitions[i].ID // There are many expression nodes in the plan tree use the original datasource // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. diff --git a/plan/stringer.go b/plan/stringer.go index b33d4224a5da2..9b51dcc62ddf5 100644 --- a/plan/stringer.go +++ b/plan/stringer.go @@ -141,7 +141,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { idxs = idxs[:last] case *DataSource: if x.isPartition { - str = fmt.Sprintf("Partition(%d)", x.partitionID) + str = fmt.Sprintf("Partition(%d)", x.physicalTableID) } else { if x.TableAsName != nil && x.TableAsName.L != "" { str = fmt.Sprintf("DataScan(%s)", x.TableAsName) diff --git a/statistics/builder.go b/statistics/builder.go index 079431b53f3fd..952a47628692f 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -156,10 +156,11 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample // AnalyzeResult is used to represent analyze result. type AnalyzeResult struct { - PhysicalID int64 // PhysicalID is the partition id for a partitioned table, otherwise, it is the table id. - Hist []*Histogram - Cms []*CMSketch - Count int64 - IsIndex int - Err error + // PhysicalTableID is the id of a partition or a table. + PhysicalTableID int64 + Hist []*Histogram + Cms []*CMSketch + Count int64 + IsIndex int + Err error } diff --git a/table/table.go b/table/table.go index e268c7c08c156..517f7dcca907c 100644 --- a/table/table.go +++ b/table/table.go @@ -149,11 +149,6 @@ type Table interface { // Meta returns TableInfo. Meta() *model.TableInfo - // GetID returns the ID of the table. - // If it is not a partition, the ID would be the tableID. - // If it is a partition, the ID would be the partitionID. - GetID() int64 - // Seek returns the handle greater or equal to h. Seek(ctx sessionctx.Context, h int64) (handle int64, found bool, err error) @@ -161,12 +156,19 @@ type Table interface { Type() Type } +// PhysicalTable is an abstraction for two kinds of table representation: partition or non-partitioned table. +// PhysicalID is a ID that can be used to construct a key ranges, all the data in the key range belongs to the corresponding PhysicalTable. +// For a non-partitioned table, its PhysicalID equals to its TableID; For a partition of a partitioned table, its PhysicalID is the partition's ID. +type PhysicalTable interface { + Table + GetPhysicalID() int64 +} + // PartitionedTable is a Table, and it has a GetPartition() method. -// GetPartition() gets the partition from a partition table by partitionID, its -// return value is a Table because partition implements the Table interface. +// GetPartition() gets the partition from a partition table by a physical table ID, type PartitionedTable interface { Table - GetPartition(partitionID int64) Table + GetPartition(physicalID int64) PhysicalTable } // TableFromMeta builds a table.Table from *model.TableInfo. diff --git a/table/tables/index.go b/table/tables/index.go index d7b71c051db5b..1781a2c16e843 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -105,13 +105,12 @@ type index struct { } // NewIndex builds a new Index object. -// id may be partition or table ID, depends on whether the table is a PartitionedTable. -func NewIndex(id int64, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) table.Index { +func NewIndex(physicalID int64, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) table.Index { index := &index{ idxInfo: indexInfo, tblInfo: tblInfo, // The prefix can't encode from tblInfo.ID, because table partition may change the id to partition id. - prefix: tablecodec.EncodeTableIndexPrefix(id, indexInfo.ID), + prefix: tablecodec.EncodeTableIndexPrefix(physicalID, indexInfo.ID), } return index } diff --git a/table/tables/partition.go b/table/tables/partition.go index 7dce282482653..8ca947f2ed85d 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -49,9 +49,9 @@ type partition struct { tableCommon } -// GetID implements table.Table GetID interface. -func (p *partition) GetID() int64 { - return p.partitionID +// GetPhysicalID implements table.Table GetPhysicalID interface. +func (p *partition) GetPhysicalID() int64 { + return p.physicalTableID } // partitionedTable implements the table.PartitionedTable interface. @@ -179,7 +179,7 @@ func (t *partitionedTable) locatePartition(ctx sessionctx.Context, pi *model.Par } // GetPartition returns a Table, which is actually a partition. -func (t *partitionedTable) GetPartition(pid int64) table.Table { +func (t *partitionedTable) GetPartition(pid int64) table.PhysicalTable { return t.partitions[pid] } @@ -245,7 +245,3 @@ func (t *partitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currDat tbl := t.GetPartition(to) return tbl.UpdateRecord(ctx, h, currData, newData, touched) } - -func (t *partitionedTable) GetID() int64 { - panic("GetID() should never be called on PartitionedTable") -} diff --git a/table/tables/tables.go b/table/tables/tables.go index 4316c522b6915..37bd31a5ce788 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -48,9 +48,8 @@ import ( // tableCommon is shared by both Table and partition. type tableCommon struct { tableID int64 - // partitionID is a unique int64 to identify a partition, it equals to tableID - // if this tableCommon struct is not a partition. - partitionID int64 + // physicalTableID is a unique int64 to identify a physical table. + physicalTableID int64 Columns []*table.Column publicColumns []*table.Column writableColumns []*table.Column @@ -59,7 +58,7 @@ type tableCommon struct { meta *model.TableInfo alloc autoid.Allocator - // recordPrefix and indexPrefix are generated using partitionID. + // recordPrefix and indexPrefix are generated using physicalTableID. recordPrefix kv.Key indexPrefix kv.Key } @@ -141,17 +140,17 @@ func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) (table.Tabl } // initTableCommon initializes a tableCommon struct. -func initTableCommon(t *tableCommon, tblInfo *model.TableInfo, partitionID int64, cols []*table.Column, alloc autoid.Allocator) { +func initTableCommon(t *tableCommon, tblInfo *model.TableInfo, physicalTableID int64, cols []*table.Column, alloc autoid.Allocator) { t.tableID = tblInfo.ID - t.partitionID = partitionID + t.physicalTableID = physicalTableID t.alloc = alloc t.meta = tblInfo t.Columns = cols t.publicColumns = t.Cols() t.writableColumns = t.WritableCols() t.writableIndices = t.WritableIndices() - t.recordPrefix = tablecodec.GenTableRecordPrefix(partitionID) - t.indexPrefix = tablecodec.GenTableIndexPrefix(partitionID) + t.recordPrefix = tablecodec.GenTableRecordPrefix(physicalTableID) + t.indexPrefix = tablecodec.GenTableIndexPrefix(physicalTableID) } // initTableIndices initializes the indices of the tableCommon. @@ -163,14 +162,14 @@ func initTableIndices(t *tableCommon) error { } // Use partition ID for index, because tableCommon may be table or partition. - idx := NewIndex(t.partitionID, tblInfo, idxInfo) + idx := NewIndex(t.physicalTableID, tblInfo, idxInfo) t.indices = append(t.indices, idx) } return nil } -func initTableCommonWithIndices(t *tableCommon, tblInfo *model.TableInfo, partitionID int64, cols []*table.Column, alloc autoid.Allocator) error { - initTableCommon(t, tblInfo, partitionID, cols, alloc) +func initTableCommonWithIndices(t *tableCommon, tblInfo *model.TableInfo, physicalTableID int64, cols []*table.Column, alloc autoid.Allocator) error { + initTableCommon(t, tblInfo, physicalTableID, cols, alloc) return errors.Trace(initTableIndices(t)) } @@ -205,9 +204,9 @@ func (t *tableCommon) Meta() *model.TableInfo { return t.meta } -// GetID implements table.Table GetID interface. -func (t *tableCommon) GetID() int64 { - return t.partitionID +// GetPhysicalID implements table.Table GetPhysicalID interface. +func (t *Table) GetPhysicalID() int64 { + return t.physicalTableID } // Cols implements table.Table Cols interface. @@ -326,8 +325,8 @@ func (t *tableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new if err = bs.SaveTo(txn); err != nil { return errors.Trace(err) } - ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.partitionID, h, nil) - ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.partitionID, h, newData) + ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.physicalTableID, h, nil) + ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.physicalTableID, h, newData) if shouldWriteBinlog(ctx) { if !t.meta.PKIsHandle { binlogColIDs = append(binlogColIDs, model.ExtraHandleID) @@ -482,7 +481,7 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHan } if !ctx.GetSessionVars().LightningMode { - ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.partitionID, recordID, r) + ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.physicalTableID, recordID, r) } if shouldWriteBinlog(ctx) { // For insert, TiDB and Binlog can use same row and schema. @@ -644,7 +643,7 @@ func (t *tableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Da return errors.Trace(err) } - ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.partitionID, h, nil) + ctx.StmtAddDirtyTableOP(table.DirtyTableDeleteRow, t.physicalTableID, h, nil) if shouldWriteBinlog(ctx) { cols := t.Cols() colIDs := make([]int64, 0, len(cols)+1) @@ -886,7 +885,7 @@ func (t *tableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetS // Seek implements table.Table Seek interface. func (t *tableCommon) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { - seekKey := tablecodec.EncodeRowKeyWithHandle(t.partitionID, h) + seekKey := tablecodec.EncodeRowKeyWithHandle(t.physicalTableID, h) iter, err := ctx.Txn().Seek(seekKey) if !iter.Valid() || !iter.Key().HasPrefix(t.RecordPrefix()) { // No more records in the table, skip to the end. diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 599c025e90376..553d97cbe5b9c 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -396,8 +396,8 @@ PARTITION BY RANGE ( id ) ( c.Assert(err, IsNil) // Insert into maxvalue partition. } -// TestPartitionGetID tests partition.GetID(). -func (ts *testSuite) TestPartitionGetID(c *C) { +// TestPartitionGetPhysicalID tests partition.GetPhysicalID(). +func (ts *testSuite) TestPartitionGetPhysicalID(c *C) { createTable1 := `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY RANGE ( id ) ( PARTITION p0 VALUES LESS THAN (6), @@ -419,6 +419,6 @@ PARTITION BY RANGE ( id ) ( for _, pd := range ps.Definitions { p := tb.(table.PartitionedTable).GetPartition(pd.ID) c.Assert(p, NotNil) - c.Assert(pd.ID, Equals, p.GetID()) + c.Assert(pd.ID, Equals, p.GetPhysicalID()) } } From 7f286f5c1212897eb27d118b3016736373ff0dc8 Mon Sep 17 00:00:00 2001 From: xiaojian cai Date: Thu, 16 Aug 2018 11:04:39 +0800 Subject: [PATCH 63/87] types: add one more space to make json pretty (#7389) --- executor/executor_test.go | 4 ++-- executor/write_test.go | 6 +++--- types/json/binary.go | 6 +++--- types/json/binary_test.go | 8 ++++---- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 2813abf9bb01a..634c5badbd6ea 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1237,7 +1237,7 @@ func (s *testSuite) TestJSON(c *C) { var result *testkit.Result result = tk.MustQuery(`select tj.a from test_json tj order by tj.id`) - result.Check(testkit.Rows(`{"a":[1,"2",{"aa":"bb"},4],"b":true}`, "null", "", "true", "3", "4", `"string"`)) + result.Check(testkit.Rows(`{"a": [1, "2", {"aa": "bb"}, 4], "b": true}`, "null", "", "true", "3", "4", `"string"`)) // Check json_type function result = tk.MustQuery(`select json_type(a) from test_json tj order by tj.id`) @@ -1481,7 +1481,7 @@ func (s *testSuite) TestGeneratedColumnRead(c *C) { tk.MustExec(`CREATE TABLE test_gc_read_cast_2( a JSON, b JSON AS (a->>'$.a'))`) tk.MustExec(`INSERT INTO test_gc_read_cast_2(a) VALUES ('{"a": "{ \\\"key\\\": \\\"\\u6d4b\\\" }"}')`) result = tk.MustQuery(`SELECT b FROM test_gc_read_cast_2`) - result.Check(testkit.Rows(`{"key":"测"}`)) + result.Check(testkit.Rows(`{"key": "测"}`)) _, err := tk.Exec(`INSERT INTO test_gc_read_cast_1 (a, b) VALUES ('{"a": "invalid"}', '$.a')`) c.Assert(err, NotNil) diff --git a/executor/write_test.go b/executor/write_test.go index f953a00a88209..09364d3baa39e 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1049,7 +1049,7 @@ func (s *testSuite) TestUpdateCastOnlyModifiedValues(c *C) { r.Check(testkit.Rows("300")) tk.MustExec(`UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'`) r = tk.MustQuery("SELECT b FROM update_with_diff_type") - r.Check(testkit.Rows(`{"a":"测试"}`)) + r.Check(testkit.Rows(`{"a": "测试"}`)) } func (s *testSuite) fillMultiTableForUpdate(tk *testkit.TestKit) { @@ -1782,13 +1782,13 @@ func (s *testSuite) TestInsertCalculatedValue(c *C) { tk.MustExec("insert into t (b) value (a->'$.a'+1)") tk.MustQuery("select * from t").Check(testkit.Rows(" ", " ")) tk.MustExec(`insert into t (a, b) value ('{"a": 1}', a->'$.a'+1)`) - tk.MustQuery("select * from t where c = 1").Check(testkit.Rows(`{"a":1} 2 1`)) + tk.MustQuery("select * from t where c = 1").Check(testkit.Rows(`{"a": 1} 2 1`)) tk.MustExec("truncate table t") tk.MustExec("insert t set b = c + 1") tk.MustQuery("select * from t").Check(testkit.Rows(" ")) tk.MustExec("truncate table t") tk.MustExec(`insert t set a = '{"a": 1}', b = c`) - tk.MustQuery("select * from t").Check(testkit.Rows(`{"a":1} 1`)) + tk.MustQuery("select * from t").Check(testkit.Rows(`{"a": 1} 1`)) tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int auto_increment key, b int)") diff --git a/types/json/binary.go b/types/json/binary.go index 0f403aa86fba0..b8ded1e0a849e 100644 --- a/types/json/binary.go +++ b/types/json/binary.go @@ -245,7 +245,7 @@ func (bj BinaryJSON) marshalArrayTo(buf []byte) ([]byte, error) { buf = append(buf, '[') for i := 0; i < elemCount; i++ { if i != 0 { - buf = append(buf, ',') + buf = append(buf, ", "...) } var err error buf, err = bj.arrayGetElem(i).marshalTo(buf) @@ -261,10 +261,10 @@ func (bj BinaryJSON) marshalObjTo(buf []byte) ([]byte, error) { buf = append(buf, '{') for i := 0; i < elemCount; i++ { if i != 0 { - buf = append(buf, ',') + buf = append(buf, ", "...) } buf = marshalStringTo(buf, bj.objectGetKey(i)) - buf = append(buf, ':') + buf = append(buf, ": "...) var err error buf, err = bj.objectGetVal(i).marshalTo(buf) if err != nil { diff --git a/types/json/binary_test.go b/types/json/binary_test.go index f2b619c511926..d03a02d9afe58 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -29,9 +29,9 @@ func TestT(t *testing.T) { func (s *testJSONSuite) TestBinaryJSONMarshalUnmarshal(c *C) { strs := []string{ - `{"a":[1,"2",{"aa":"bb"},4,null],"b":true,"c":null}`, - `{"aaaaaaaaaaa":[1,"2",{"aa":"bb"},4.1],"bbbbbbbbbb":true,"ccccccccc":"d"}`, - `[{"a":1,"b":true},3,3.5,"hello, world",null,true]`, + `{"a": [1, "2", {"aa": "bb"}, 4, null], "b": true, "c": null}`, + `{"aaaaaaaaaaa": [1, "2", {"aa": "bb"}, 4.1], "bbbbbbbbbb": true, "ccccccccc": "d"}`, + `[{"a": 1, "b": true}, 3, 3.5, "hello, world", null, true]`, } for _, str := range strs { parsedBJ := mustParseBinaryFromString(c, str) @@ -115,7 +115,7 @@ func (s *testJSONSuite) TestBinaryJSONUnquote(c *C) { {j: "\"\\u4f60\"", unquoted: "你"}, {j: `true`, unquoted: "true"}, {j: `null`, unquoted: "null"}, - {j: `{"a": [1, 2]}`, unquoted: `{"a":[1,2]}`}, + {j: `{"a": [1, 2]}`, unquoted: `{"a": [1, 2]}`}, {j: `"\""`, unquoted: `"`}, {j: `"'"`, unquoted: `'`}, {j: `"''"`, unquoted: `''`}, From 1ebf316bd414f5aa55493e152b5991b08873d63c Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 16 Aug 2018 11:40:10 +0800 Subject: [PATCH 64/87] domain: handling assignments in server info (#7412) --- domain/domain.go | 1 + domain/info.go | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 7f80aa0baf5f0..1f6a5a341eac6 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -366,6 +366,7 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) { } do.SchemaValidator.Restart() case <-do.info.Done(): + log.Info("[ddl] reload schema in loop, server info syncer need restart") do.info.Restart(context.Background()) case <-do.exit: return diff --git a/domain/info.go b/domain/info.go index 6ab5a5f5d5bdb..3e066b0eacf59 100644 --- a/domain/info.go +++ b/domain/info.go @@ -160,12 +160,13 @@ func (is *InfoSyncer) newSessionAndStoreServerInfo(ctx context.Context, retryCnt if is.etcdCli == nil { return nil } - var err error logPrefix := fmt.Sprintf("[Info-syncer] %s", is.serverInfoPath) - is.session, err = owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, InfoSessionTTL) + session, err := owner.NewSession(ctx, logPrefix, is.etcdCli, retryCnt, InfoSessionTTL) if err != nil { return errors.Trace(err) } + is.session = session + err = is.storeServerInfo(ctx) return errors.Trace(err) } From bbec68304e794a1a5e219667ce2ba6ee0e28ce22 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 16 Aug 2018 13:28:50 +0800 Subject: [PATCH 65/87] executor,table: support replace operation for table partition (#7309) --- executor/batch_checker.go | 137 ++++++++++++++++------------ executor/replace.go | 45 ++++----- executor/write_test.go | 186 ++++++++++++++++++++++++++++++++------ table/table.go | 1 + table/tables/partition.go | 20 +++- 5 files changed, 279 insertions(+), 110 deletions(-) diff --git a/executor/batch_checker.go b/executor/batch_checker.go index 922b451042d80..ff132e191c500 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -39,6 +39,8 @@ type toBeCheckedRow struct { rowValue []byte handleKey *keyValueWithDupInfo uniqueKeys []*keyValueWithDupInfo + // The table or partition this row belongs to. + t table.Table } type batchChecker struct { @@ -49,11 +51,7 @@ type batchChecker struct { } // batchGetOldValues gets the values of storage in batch. -func (b *batchChecker) batchGetOldValues(ctx sessionctx.Context, t table.Table, handles []int64) error { - batchKeys := make([]kv.Key, 0, len(handles)) - for _, handle := range handles { - batchKeys = append(batchKeys, t.RecordKey(handle)) - } +func (b *batchChecker) batchGetOldValues(ctx sessionctx.Context, batchKeys []kv.Key) error { values, err := kv.BatchGetValues(ctx.Txn(), batchKeys) if err != nil { return errors.Trace(err) @@ -103,65 +101,83 @@ func (b *batchChecker) getKeysNeedCheck(ctx sessionctx.Context, t table.Table, r } } + var err error for _, row := range rows { - var handleKey *keyValueWithDupInfo - uniqueKeys := make([]*keyValueWithDupInfo, 0, nUnique) - newRowValue, err := b.encodeNewRow(ctx, t, row) + toBeCheckRows, err = b.getKeysNeedCheckOneRow(ctx, t, row, nUnique, handleCol, toBeCheckRows) if err != nil { return nil, errors.Trace(err) } - // Append record keys and errors. - if t.Meta().PKIsHandle { - handle := row[handleCol.Offset].GetInt64() - handleKey = &keyValueWithDupInfo{ - newKV: keyValue{ - key: t.RecordKey(handle), - value: newRowValue, - }, - dupErr: kv.ErrKeyExists.FastGen("Duplicate entry '%d' for key 'PRIMARY'", handle), - } + } + return toBeCheckRows, nil +} + +func (b *batchChecker) getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.Datum, nUnique int, handleCol *table.Column, result []toBeCheckedRow) ([]toBeCheckedRow, error) { + var err error + if p, ok := t.(table.PartitionedTable); ok { + t, err = p.GetPartitionByRow(ctx, row) + if err != nil { + return nil, errors.Trace(err) } + } - // append unique keys and errors - for _, v := range t.WritableIndices() { - if !v.Meta().Unique { - continue - } - colVals, err1 := v.FetchValues(row, nil) - if err1 != nil { - return nil, errors.Trace(err1) - } - // Pass handle = 0 to GenIndexKey, - // due to we only care about distinct key. - key, distinct, err1 := v.GenIndexKey(ctx.GetSessionVars().StmtCtx, - colVals, 0, nil) - if err1 != nil { - return nil, errors.Trace(err1) - } - // Skip the non-distinct keys. - if !distinct { - continue - } - colValStr, err1 := types.DatumsToString(colVals, false) - if err1 != nil { - return nil, errors.Trace(err1) - } - uniqueKeys = append(uniqueKeys, &keyValueWithDupInfo{ - newKV: keyValue{ - key: key, - }, - dupErr: kv.ErrKeyExists.FastGen("Duplicate entry '%s' for key '%s'", - colValStr, v.Meta().Name), - }) + var handleKey *keyValueWithDupInfo + uniqueKeys := make([]*keyValueWithDupInfo, 0, nUnique) + newRowValue, err := b.encodeNewRow(ctx, t, row) + if err != nil { + return nil, errors.Trace(err) + } + // Append record keys and errors. + if handleCol != nil { + handle := row[handleCol.Offset].GetInt64() + handleKey = &keyValueWithDupInfo{ + newKV: keyValue{ + key: t.RecordKey(handle), + value: newRowValue, + }, + dupErr: kv.ErrKeyExists.FastGen("Duplicate entry '%d' for key 'PRIMARY'", handle), + } + } + + // append unique keys and errors + for _, v := range t.WritableIndices() { + if !v.Meta().Unique { + continue + } + colVals, err1 := v.FetchValues(row, nil) + if err1 != nil { + return nil, errors.Trace(err1) } - toBeCheckRows = append(toBeCheckRows, toBeCheckedRow{ - row: row, - rowValue: newRowValue, - handleKey: handleKey, - uniqueKeys: uniqueKeys, + // Pass handle = 0 to GenIndexKey, + // due to we only care about distinct key. + key, distinct, err1 := v.GenIndexKey(ctx.GetSessionVars().StmtCtx, + colVals, 0, nil) + if err1 != nil { + return nil, errors.Trace(err1) + } + // Skip the non-distinct keys. + if !distinct { + continue + } + colValStr, err1 := types.DatumsToString(colVals, false) + if err1 != nil { + return nil, errors.Trace(err1) + } + uniqueKeys = append(uniqueKeys, &keyValueWithDupInfo{ + newKV: keyValue{ + key: key, + }, + dupErr: kv.ErrKeyExists.FastGen("Duplicate entry '%s' for key '%s'", + colValStr, v.Meta().Name), }) } - return toBeCheckRows, nil + result = append(result, toBeCheckedRow{ + row: row, + rowValue: newRowValue, + handleKey: handleKey, + uniqueKeys: uniqueKeys, + t: t, + }) + return result, nil } // batchGetInsertKeys uses batch-get to fetch all key-value pairs to be checked for ignore or duplicate key update. @@ -205,8 +221,8 @@ func (b *batchChecker) initDupOldRowFromHandleKey() { } } -func (b *batchChecker) initDupOldRowFromUniqueKey(ctx sessionctx.Context, t table.Table, newRows [][]types.Datum) error { - handles := make([]int64, 0, len(newRows)) +func (b *batchChecker) initDupOldRowFromUniqueKey(ctx sessionctx.Context, newRows [][]types.Datum) error { + batchKeys := make([]kv.Key, 0, len(newRows)) for _, r := range b.toBeCheckedRows { for _, uk := range r.uniqueKeys { if val, found := b.dupKVs[string(uk.newKV.key)]; found { @@ -214,18 +230,18 @@ func (b *batchChecker) initDupOldRowFromUniqueKey(ctx sessionctx.Context, t tabl if err != nil { return errors.Trace(err) } - handles = append(handles, handle) + batchKeys = append(batchKeys, r.t.RecordKey(handle)) } } } - return errors.Trace(b.batchGetOldValues(ctx, t, handles)) + return errors.Trace(b.batchGetOldValues(ctx, batchKeys)) } // initDupOldRowValue initializes dupOldRowValues which contain the to-be-updated rows from storage. func (b *batchChecker) initDupOldRowValue(ctx sessionctx.Context, t table.Table, newRows [][]types.Datum) error { b.dupOldRowValues = make(map[string][]byte, len(newRows)) b.initDupOldRowFromHandleKey() - return errors.Trace(b.initDupOldRowFromUniqueKey(ctx, t, newRows)) + return errors.Trace(b.initDupOldRowFromUniqueKey(ctx, newRows)) } // fillBackKeys fills the updated key-value pair to the dupKeyValues for further check. @@ -251,6 +267,7 @@ func (b *batchChecker) deleteDupKeys(row toBeCheckedRow) { } // getOldRow gets the table record row from storage for batch check. +// t could be a normal table or a partition, but it must not be a PartitionedTable. func (b *batchChecker) getOldRow(ctx sessionctx.Context, t table.Table, handle int64) ([]types.Datum, error) { oldValue, ok := b.dupOldRowValues[string(t.RecordKey(handle))] if !ok { diff --git a/executor/replace.go b/executor/replace.go index 1f00648c67293..595c473e8a34b 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -48,8 +48,9 @@ func (e *ReplaceExec) Open(ctx context.Context) error { // removeRow removes the duplicate row and cleanup its keys in the key-value map, // but if the to-be-removed row equals to the to-be-added row, no remove or add things to do. -func (e *ReplaceExec) removeRow(handle int64, newRow []types.Datum) (bool, error) { - oldRow, err := e.getOldRow(e.ctx, e.Table, handle) +func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { + newRow := r.row + oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { return false, errors.Trace(err) } @@ -61,14 +62,15 @@ func (e *ReplaceExec) removeRow(handle int64, newRow []types.Datum) (bool, error e.ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) return true, nil } - err = e.Table.RemoveRecord(e.ctx, handle, oldRow) + + err = r.t.RemoveRecord(e.ctx, handle, oldRow) if err != nil { return false, errors.Trace(err) } e.ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) // Cleanup keys map, because the record was removed. - cleanupRows, err := e.getKeysNeedCheck(e.ctx, e.Table, [][]types.Datum{oldRow}) + cleanupRows, err := e.getKeysNeedCheck(e.ctx, r.t, [][]types.Datum{oldRow}) if err != nil { return false, errors.Trace(err) } @@ -93,25 +95,24 @@ func (e *ReplaceExec) addRow(row []types.Datum) (int64, error) { // replaceRow removes all duplicate rows for one row, then inserts it. func (e *ReplaceExec) replaceRow(r toBeCheckedRow) error { - // Keep on removing duplicated rows. - for { - if r.handleKey != nil { - if _, found := e.dupKVs[string(r.handleKey.newKV.key)]; found { - handle, err := tablecodec.DecodeRowKey(r.handleKey.newKV.key) - if err != nil { - return errors.Trace(err) - } - rowUnchanged, err := e.removeRow(handle, r.row) - if err != nil { - return errors.Trace(err) - } - if rowUnchanged { - return nil - } - continue + if r.handleKey != nil { + if _, found := e.dupKVs[string(r.handleKey.newKV.key)]; found { + handle, err := tablecodec.DecodeRowKey(r.handleKey.newKV.key) + if err != nil { + return errors.Trace(err) + } + rowUnchanged, err := e.removeRow(handle, r) + if err != nil { + return errors.Trace(err) + } + if rowUnchanged { + return nil } } + } + // Keep on removing duplicated rows. + for { rowUnchanged, foundDupKey, err := e.removeIndexRow(r) if err != nil { return errors.Trace(err) @@ -130,7 +131,7 @@ func (e *ReplaceExec) replaceRow(r toBeCheckedRow) error { if err != nil { return errors.Trace(err) } - e.fillBackKeys(e.Table, r, newHandle) + e.fillBackKeys(r.t, r, newHandle) return nil } @@ -147,7 +148,7 @@ func (e *ReplaceExec) removeIndexRow(r toBeCheckedRow) (bool, bool, error) { if err != nil { return false, found, errors.Trace(err) } - rowUnchanged, err := e.removeRow(handle, r.row) + rowUnchanged, err := e.removeRow(handle, r) if err != nil { return false, found, errors.Trace(err) } diff --git a/executor/write_test.go b/executor/write_test.go index 09364d3baa39e..aa8a7649b12e1 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -777,6 +777,139 @@ func (s *testSuite) TestReplace(c *C) { r.Check(testkit.Rows("1 1")) } +func (s *testSuite) TestPartitionedTableReplace(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_enable_table_partition=1") + testSQL := `drop table if exists replace_test; + create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) + partition by range (id) ( + PARTITION p0 VALUES LESS THAN (3), + PARTITION p1 VALUES LESS THAN (5), + PARTITION p2 VALUES LESS THAN (7), + PARTITION p3 VALUES LESS THAN (9));` + tk.MustExec(testSQL) + testSQL = `replace replace_test (c1) values (1),(2),(NULL);` + tk.MustExec(testSQL) + + errReplaceSQL := `replace replace_test (c1) values ();` + tk.MustExec("begin") + _, err := tk.Exec(errReplaceSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + errReplaceSQL = `replace replace_test (xxx) values (3);` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + errReplaceSQL = `replace replace_test_xxx (c1) values ();` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + replaceSetSQL := `replace replace_test set c1 = 3;` + tk.MustExec(replaceSetSQL) + + errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSetSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + errReplaceSetSQL = `replace replace_test set xxx = 6;` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSetSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + tk.MustExec(`drop table if exists replace_test_1`) + tk.MustExec(`create table replace_test_1 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (6), + PARTITION p2 VALUES LESS THAN (8), + PARTITION p3 VALUES LESS THAN (10), + PARTITION p4 VALUES LESS THAN (100))`) + tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) + + tk.MustExec(`drop table if exists replace_test_2`) + tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (50), + PARTITION p2 VALUES LESS THAN (100), + PARTITION p3 VALUES LESS THAN (300))`) + tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) + + errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` + tk.MustExec("begin") + _, err = tk.Exec(errReplaceSelectSQL) + c.Assert(err, NotNil) + tk.MustExec("rollback") + + tk.MustExec(`drop table if exists replace_test_3`) + replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11))` + tk.MustExec(replaceUniqueIndexSQL) + replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` + tk.MustExec(replaceUniqueIndexSQL) + replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` + tk.MustExec(replaceUniqueIndexSQL) + c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + replaceUniqueIndexSQL = `replace into replace_test_3 set c1=8, c2=8;` + tk.MustExec(replaceUniqueIndexSQL) + c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) + + replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` + tk.MustExec(replaceUniqueIndexSQL) + replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` + tk.MustExec(replaceUniqueIndexSQL) + c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + + replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11));` + tk.MustExec(`drop table if exists replace_test_4`) + tk.MustExec(replaceUniqueIndexSQL) + replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` + tk.MustExec(replaceUniqueIndexSQL) + replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` + tk.MustExec(replaceUniqueIndexSQL) + c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + + replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11));` + tk.MustExec(replacePrimaryKeySQL) + replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` + tk.MustExec(replacePrimaryKeySQL) + replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` + tk.MustExec(replacePrimaryKeySQL) + c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + + issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (200))` + tk.MustExec(issue989SQL) + issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` + tk.MustExec(issue989SQL) + issue989SQL = `replace into tIssue989(a, b) values (111, 2);` + tk.MustExec(issue989SQL) + r := tk.MustQuery("select * from tIssue989;") + r.Check(testkit.Rows("111 2")) +} + func (s *testSuite) TestUpdate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -924,11 +1057,11 @@ func (s *testSuite) TestPartitionedTableUpdate(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(`create table t (id int not null default 1, name varchar(255)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) tk.MustExec(`insert INTO t VALUES (1, "hello");`) tk.CheckExecResult(1, 0) @@ -956,11 +1089,11 @@ func (s *testSuite) TestPartitionedTableUpdate(c *C) { // table option is auto-increment tk.MustExec("drop table if exists t;") tk.MustExec(`create table t (id int not null auto_increment, name varchar(255), primary key(id)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) tk.MustExec("insert into t(name) values ('aa')") tk.MustExec("update t set id = 8 where name = 'aa'") @@ -975,11 +1108,11 @@ func (s *testSuite) TestPartitionedTableUpdate(c *C) { // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. tk.MustExec("drop table if exists t;") tk.MustExec(`create table t (id int, name int unique) - PARTITION BY RANGE ( name ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) + PARTITION BY RANGE ( name ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))`) tk.MustExec("insert t values (1, 1), (2, 2);") _, err = tk.Exec("update t set name = 1 where id = 2") c.Assert(err, NotNil) @@ -988,9 +1121,9 @@ func (s *testSuite) TestPartitionedTableUpdate(c *C) { // test update ignore for pimary key tk.MustExec("drop table if exists t;") tk.MustExec(`create table t(a bigint, primary key (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11))`) tk.MustExec("insert into t values (5)") tk.MustExec("insert into t values (7)") _, err = tk.Exec("update ignore t set a = 5 where a = 7;") @@ -1008,9 +1141,9 @@ func (s *testSuite) TestPartitionedTableUpdate(c *C) { // test update ignore for unique key tk.MustExec("drop table if exists t;") tk.MustExec(`create table t(a bigint, unique key I_uniq (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11))`) tk.MustExec("insert into t values (5)") tk.MustExec("insert into t values (7)") _, err = tk.Exec("update ignore t set a = 5 where a = 7;") @@ -1160,12 +1293,11 @@ func (s *testSuite) TestDelete(c *C) { func (s *testSuite) TestPartitionedTableDelete(c *C) { createTable := `CREATE TABLE test.t (id int not null default 1, name varchar(255), index(id)) -PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21) -)` + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21))` tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@session.tidb_enable_table_partition=1") diff --git a/table/table.go b/table/table.go index 517f7dcca907c..ac99df34f397e 100644 --- a/table/table.go +++ b/table/table.go @@ -169,6 +169,7 @@ type PhysicalTable interface { type PartitionedTable interface { Table GetPartition(physicalID int64) PhysicalTable + GetPartitionByRow(sessionctx.Context, []types.Datum) (Table, error) } // TableFromMeta builds a table.Table from *model.TableInfo. diff --git a/table/tables/partition.go b/table/tables/partition.go index 8ca947f2ed85d..792ce7ad46494 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -159,18 +159,27 @@ func partitionRecordKey(pid int64, handle int64) kv.Key { // locatePartition returns the partition ID of the input record. func (t *partitionedTable) locatePartition(ctx sessionctx.Context, pi *model.PartitionInfo, r []types.Datum) (int64, error) { var err error + var isNull bool partitionExprs := t.partitionExpr.UpperBounds idx := sort.Search(len(partitionExprs), func(i int) bool { var ret int64 - ret, _, err = partitionExprs[i].EvalInt(ctx, chunk.MutRowFromDatums(r).ToRow()) + ret, isNull, err = partitionExprs[i].EvalInt(ctx, chunk.MutRowFromDatums(r).ToRow()) if err != nil { return true // Break the search. } + if isNull { + // If the column value used to determine the partition is NULL, the row is inserted into the lowest partition. + // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-handling-nulls.html + return true // Break the search. + } return ret > 0 }) if err != nil { return 0, errors.Trace(err) } + if isNull { + idx = 0 + } if idx < 0 || idx >= len(partitionExprs) { // The data does not belong to any of the partition? return 0, errors.Trace(table.ErrTrgInvalidCreationCtx) @@ -183,6 +192,15 @@ func (t *partitionedTable) GetPartition(pid int64) table.PhysicalTable { return t.partitions[pid] } +// GetPartitionByRow returns a Table, which is actually a Partition. +func (t *partitionedTable) GetPartitionByRow(ctx sessionctx.Context, r []types.Datum) (table.Table, error) { + pid, err := t.locatePartition(ctx, t.Meta().GetPartitionInfo(), r) + if err != nil { + return nil, errors.Trace(err) + } + return t.partitions[pid], nil +} + // AddRecord implements the AddRecord method for the table.Table interface. func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error) { partitionInfo := t.meta.GetPartitionInfo() From 5fbf5ce613bedea306eef6909d5a5bbd929f6edc Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Thu, 16 Aug 2018 14:51:52 +0800 Subject: [PATCH 66/87] util/mock: optimize mock.NewContext (#7411) --- sessionctx/variable/mock_globalaccessor.go | 20 ++++++++------------ util/mock/mock_test.go | 7 +++++++ 2 files changed, 15 insertions(+), 12 deletions(-) diff --git a/sessionctx/variable/mock_globalaccessor.go b/sessionctx/variable/mock_globalaccessor.go index 568e99789abae..24ab573eb2610 100644 --- a/sessionctx/variable/mock_globalaccessor.go +++ b/sessionctx/variable/mock_globalaccessor.go @@ -15,32 +15,28 @@ package variable // MockGlobalAccessor implements GlobalVarAccessor interface. it's used in tests type MockGlobalAccessor struct { - vars map[string]string } // NewMockGlobalAccessor implements GlobalVarAccessor interface. func NewMockGlobalAccessor() *MockGlobalAccessor { - m := &MockGlobalAccessor{ - vars: make(map[string]string), - } - for name, val := range SysVars { - m.vars[name] = val.Value - } - return m + return new(MockGlobalAccessor) } // GetGlobalSysVar implements GlobalVarAccessor.GetGlobalSysVar interface. func (m *MockGlobalAccessor) GetGlobalSysVar(name string) (string, error) { - return m.vars[name], nil + v, ok := SysVars[name] + if ok { + return v.Value, nil + } + return "", nil } // SetGlobalSysVar implements GlobalVarAccessor.SetGlobalSysVar interface. func (m *MockGlobalAccessor) SetGlobalSysVar(name string, value string) error { - m.vars[name] = value - return nil + panic("not supported") } // GetAllSysVars implements GlobalVarAccessor.GetAllSysVars interface. func (m *MockGlobalAccessor) GetAllSysVars() (map[string]string, error) { - return m.vars, nil + panic("not supported") } diff --git a/util/mock/mock_test.go b/util/mock/mock_test.go index b5708e8d639ae..701c65615c045 100644 --- a/util/mock/mock_test.go +++ b/util/mock/mock_test.go @@ -50,3 +50,10 @@ func (s *testMockSuite) TestContext(c *C) { v = ctx.Value(contextKey) c.Assert(v, IsNil) } + +func BenchmarkNewContext(b *testing.B) { + b.ReportAllocs() + for i := 0; i < b.N; i++ { + NewContext() + } +} From 3d7d81c86886ab538060a6df6187ea1098c13499 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 16 Aug 2018 15:15:01 +0800 Subject: [PATCH 67/87] fix etcd watcher goroutine leak test failure (#7408) --- ddl/table_split_test.go | 2 + owner/fail_test.go | 47 ++++--- privilege/privileges/cache_test.go | 9 +- session/bench_test.go | 195 +++++++++++++++++++++-------- session/tidb_test.go | 5 +- store/tikv/2pc_test.go | 1 + store/tikv/lock_resolver.go | 5 +- 7 files changed, 189 insertions(+), 75 deletions(-) diff --git a/ddl/table_split_test.go b/ddl/table_split_test.go index 4ab9b0132cd66..b4a2d67e4b6d3 100644 --- a/ddl/table_split_test.go +++ b/ddl/table_split_test.go @@ -31,11 +31,13 @@ var _ = Suite(&testDDLTableSplitSuite{}) func (s *testDDLTableSplitSuite) TestTableSplit(c *C) { store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) + defer store.Close() session.SetSchemaLease(0) session.SetStatsLease(0) ddl.EnableSplitTableRegion = true dom, err := session.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() ddl.EnableSplitTableRegion = false infoSchema := dom.InfoSchema() c.Assert(infoSchema, NotNil) diff --git a/owner/fail_test.go b/owner/fail_test.go index 99d7453518878..acc94606441ec 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -66,21 +66,36 @@ var ( func (s *testSuite) TestFailNewSession(c *C) { defer testleak.AfterTest(c)() - cli, err := clientv3.New(clientv3.Config{ - Endpoints: endpoints, - DialTimeout: dialTimeout, - }) - gofail.Enable("github.com/pingcap/tidb/owner/closeClient", `return(true)`) - _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) - isContextDone := terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) - c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) + func() { + cli, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + DialTimeout: dialTimeout, + }) + defer func() { + if cli != nil { + cli.Close() + } + }() + gofail.Enable("github.com/pingcap/tidb/owner/closeClient", `return(true)`) + _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) + isContextDone := terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) + c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) + }() + + func() { + cli, err := clientv3.New(clientv3.Config{ + Endpoints: endpoints, + DialTimeout: dialTimeout, + }) + defer func() { + if cli != nil { + cli.Close() + } + }() + gofail.Enable("github.com/pingcap/tidb/owner/closeGrpc", `return(true)`) + _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) + isContextDone := terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) + c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) + }() - cli, err = clientv3.New(clientv3.Config{ - Endpoints: endpoints, - DialTimeout: dialTimeout, - }) - gofail.Enable("github.com/pingcap/tidb/owner/closeGrpc", `return(true)`) - _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) - isContextDone = terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) - c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) } diff --git a/privilege/privileges/cache_test.go b/privilege/privileges/cache_test.go index 636ce84352e3f..723373e80ffa3 100644 --- a/privilege/privileges/cache_test.go +++ b/privilege/privileges/cache_test.go @@ -41,7 +41,7 @@ func (s *testCacheSuite) SetUpSuite(c *C) { s.store = store } -func (s *testCacheSuite) TearDown(c *C) { +func (s *testCacheSuite) TearDownSuit(c *C) { s.domain.Close() s.store.Close() } @@ -187,11 +187,14 @@ func (s *testCacheSuite) TestCaseInsensitive(c *C) { func (s *testCacheSuite) TestAbnormalMySQLTable(c *C) { store, err := mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + defer store.Close() session.SetSchemaLease(0) session.SetStatsLease(0) + + dom, err := session.BootstrapSession(store) c.Assert(err, IsNil) - s.domain, err = session.BootstrapSession(store) - c.Assert(err, IsNil) + defer dom.Close() se, err := session.CreateSession4Test(store) c.Assert(err, IsNil) diff --git a/session/bench_test.go b/session/bench_test.go index 777876e570c02..f19dd711e257f 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -20,6 +20,8 @@ import ( "time" "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -28,12 +30,12 @@ import ( var smallCount = 100 var bigCount = 10000 -func prepareBenchSession() Session { +func prepareBenchSession() (Session, *domain.Domain, kv.Storage) { store, err := mockstore.NewMockTikvStore() if err != nil { log.Fatal(err) } - _, err = BootstrapSession(store) + domain, err := BootstrapSession(store) if err != nil { log.Fatal(err) } @@ -43,7 +45,7 @@ func prepareBenchSession() Session { log.Fatal(err) } mustExecute(se, "use test") - return se + return se, domain, store } func prepareBenchData(se Session, colType string, valueFormat string, valueCount int) { @@ -98,9 +100,13 @@ func readResult(ctx context.Context, rs ast.RecordSet, count int) { func BenchmarkBasic(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() - b.StartTimer() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select 1") if err != nil { @@ -108,14 +114,19 @@ func BenchmarkBasic(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkTableScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%v", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t") if err != nil { @@ -123,14 +134,19 @@ func BenchmarkTableScan(b *testing.B) { } readResult(ctx, rs[0], smallCount) } + b.StopTimer() } func BenchmarkExplainTableScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%v", 0) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "explain select * from t") if err != nil { @@ -138,14 +154,19 @@ func BenchmarkExplainTableScan(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkTableLookup(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%d", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where pk = 64") if err != nil { @@ -153,14 +174,19 @@ func BenchmarkTableLookup(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkExplainTableLookup(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%d", 0) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "explain select * from t where pk = 64") if err != nil { @@ -168,14 +194,19 @@ func BenchmarkExplainTableLookup(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkStringIndexScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "varchar(255)", "'hello %d'", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col > 'hello'") if err != nil { @@ -183,14 +214,19 @@ func BenchmarkStringIndexScan(b *testing.B) { } readResult(ctx, rs[0], smallCount) } + b.StopTimer() } func BenchmarkExplainStringIndexScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "varchar(255)", "'hello %d'", 0) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "explain select * from t where col > 'hello'") if err != nil { @@ -198,14 +234,19 @@ func BenchmarkExplainStringIndexScan(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkStringIndexLookup(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "varchar(255)", "'hello %d'", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col = 'hello 64'") if err != nil { @@ -213,14 +254,19 @@ func BenchmarkStringIndexLookup(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkIntegerIndexScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%v", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col >= 0") if err != nil { @@ -228,14 +274,19 @@ func BenchmarkIntegerIndexScan(b *testing.B) { } readResult(ctx, rs[0], smallCount) } + b.StopTimer() } func BenchmarkIntegerIndexLookup(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "int", "%v", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col = 64") if err != nil { @@ -243,14 +294,19 @@ func BenchmarkIntegerIndexLookup(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkDecimalIndexScan(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "decimal(32,6)", "%v.1234", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col >= 0") if err != nil { @@ -258,14 +314,19 @@ func BenchmarkDecimalIndexScan(b *testing.B) { } readResult(ctx, rs[0], smallCount) } + b.StopTimer() } func BenchmarkDecimalIndexLookup(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareBenchData(se, "decimal(32,6)", "%v.1234", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t where col = 64.1234") if err != nil { @@ -273,36 +334,51 @@ func BenchmarkDecimalIndexLookup(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } func BenchmarkInsertWithIndex(b *testing.B) { - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() mustExecute(se, "drop table if exists t") mustExecute(se, "create table t (pk int primary key, col int, index idx (col))") - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { mustExecute(se, fmt.Sprintf("insert t values (%d, %d)", i, i)) } + b.StopTimer() } func BenchmarkInsertNoIndex(b *testing.B) { - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() mustExecute(se, "drop table if exists t") mustExecute(se, "create table t (pk int primary key, col int)") - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { mustExecute(se, fmt.Sprintf("insert t values (%d, %d)", i, i)) } + b.StopTimer() } func BenchmarkSort(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareSortBenchData(se, "int", "%v", bigCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t order by col limit 50") if err != nil { @@ -310,14 +386,19 @@ func BenchmarkSort(b *testing.B) { } readResult(ctx, rs[0], 50) } + b.StopTimer() } func BenchmarkJoin(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareJoinBenchData(se, "int", "%v", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t a join t b on a.col = b.col") if err != nil { @@ -325,14 +406,19 @@ func BenchmarkJoin(b *testing.B) { } readResult(ctx, rs[0], smallCount) } + b.StopTimer() } func BenchmarkJoinLimit(b *testing.B) { ctx := context.Background() - b.StopTimer() - se := prepareBenchSession() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + st.Close() + do.Close() + }() prepareJoinBenchData(se, "int", "%v", smallCount) - b.StartTimer() + b.ResetTimer() for i := 0; i < b.N; i++ { rs, err := se.Execute(ctx, "select * from t a join t b on a.col = b.col limit 1") if err != nil { @@ -340,4 +426,5 @@ func BenchmarkJoinLimit(b *testing.B) { } readResult(ctx, rs[0], 1) } + b.StopTimer() } diff --git a/session/tidb_test.go b/session/tidb_test.go index 13eb54fed367c..776a4884daeba 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -116,7 +116,10 @@ func (s *testMainSuite) TestTrimSQL(c *C) { func (s *testMainSuite) TestRetryOpenStore(c *C) { begin := time.Now() RegisterStore("dummy", &brokenStore{}) - _, err := newStoreWithRetry("dummy://dummy-store", 3) + store, err := newStoreWithRetry("dummy://dummy-store", 3) + if store != nil { + defer store.Close() + } c.Assert(err, NotNil) elapse := time.Since(begin) c.Assert(uint64(elapse), GreaterEqual, uint64(3*time.Second)) diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index a2da94956c9b2..7aa9bf5d6d9f7 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -52,6 +52,7 @@ func (s *testCommitterSuite) SetUpTest(c *C) { func (s *testCommitterSuite) TearDownSuite(c *C) { CommitMaxBackoff = 20000 + s.store.Close() s.OneByOneSuite.TearDownSuite(c) } diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 62710fd544338..68daded4baca2 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -52,8 +52,11 @@ func newLockResolver(store Storage) *LockResolver { return r } +// NewLockResolver is exported for other pkg to use, suppress unused warning. +var _ = NewLockResolver + // NewLockResolver creates a LockResolver. -// It is exported for other services to use. For instance, binlog service needs +// It is exported for other pkg to use. For instance, binlog service needs // to determine a transaction's commit state. func NewLockResolver(etcdAddrs []string, security config.Security) (*LockResolver, error) { pdCli, err := pd.NewClient(etcdAddrs, pd.SecurityOption{ From 6e7d75299dd8426d40f297b29eddab52f45c6613 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 16 Aug 2018 17:05:59 +0800 Subject: [PATCH 68/87] test: refine test (#7414) --- ddl/db_integration_test.go | 12 +++---- ddl/schema_test.go | 2 +- executor/aggregate_test.go | 46 +++++++++++++-------------- executor/ddl_test.go | 4 +-- executor/executor_test.go | 10 +++--- executor/prepared_test.go | 2 +- executor/set_test.go | 22 ++++++------- executor/simple_test.go | 8 ++--- executor/statement_context_test.go | 2 +- expression/builtin_cast_test.go | 12 +++---- expression/builtin_encryption_test.go | 2 +- expression/builtin_string_test.go | 4 +-- expression/integration_test.go | 8 ++--- kv/union_store_test.go | 2 +- server/conn_stmt_test.go | 2 +- session/session_test.go | 20 ++++++------ table/tables/index_test.go | 6 ++-- types/convert_test.go | 22 ++++++------- types/etc_test.go | 2 +- util/codec/codec_test.go | 4 +-- 20 files changed, 96 insertions(+), 96 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 4caab8c4fba8e..e4a54fccdf1c6 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -72,11 +72,11 @@ func (s *testIntegrationSuite) TestInvalidDefault(c *C) { _, err := tk.Exec("create table t(c1 decimal default 1.7976931348623157E308)") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("create table t( c1 varchar(2) default 'TiDB');") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue, Commentf("err %v", err)) } // for issue #3848 @@ -87,15 +87,15 @@ func (s *testIntegrationSuite) TestInvalidNameWhenCreateTable(c *C) { _, err := tk.Exec("create table t(xxx.t.a bigint)") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue) + c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("create table t(test.tttt.a bigint)") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongTableName), IsTrue) + c.Assert(terror.ErrorEqual(err, ddl.ErrWrongTableName), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("create table t(t.tttt.a bigint)") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue) + c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue, Commentf("err %v", err)) } // for issue #6879 @@ -112,7 +112,7 @@ func (s *testIntegrationSuite) TestCreateTableIfNotExists(c *C) { warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() c.Assert(len(warnings), GreaterEqual, 1) lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) c.Assert(lastWarn.Level, Equals, stmtctx.WarnLevelNote) // Test duplicate create-table without `LIKE` clause diff --git a/ddl/schema_test.go b/ddl/schema_test.go index 0e1e833018709..b57d5f7343725 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -173,7 +173,7 @@ func (s *testSchemaSuite) TestSchema(c *C) { BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), IsTrue) + c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), IsTrue, Commentf("err %v", err)) // Drop a database without a table. dbInfo1 := testSchemaInfo(c, d, "test1") diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f096dfed3066c..b02f691a3dcea 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -442,39 +442,39 @@ func (s *testSuite) TestOnlyFullGroupBy(c *C) { // test incompatible with sql_mode = ONLY_FULL_GROUP_BY var err error _, err = tk.Exec("select * from t group by d") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select b-c from t group by b+c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select (b-c)*(b+c), min(a) from t group by b+c, b-c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select b between c and d from t group by b,c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select case b when 1 then c when 2 then d else d end from t group by b,c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select c > (select b from t) from t group by b") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select c is null from t group by b") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select c is true from t group by b") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select (c+b)*d from t group by c,d") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select b in (c,d) from t group by b,c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select b like '%a' from t group by c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select c REGEXP '1.*' from t group by b") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select -b from t group by c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select a, max(b) from t") - c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select sum(a)+b from t") - c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select count(b), c from t") - c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select distinct a, b, count(a) from t") - c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) // test compatible with sql_mode = ONLY_FULL_GROUP_BY tk.MustQuery("select a from t group by a,b,c") tk.MustQuery("select b from t group by b") @@ -503,7 +503,7 @@ func (s *testSuite) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select * from t group by b,d") // test functional depend on a unique null column _, err = tk.Exec("select * from t group by b,c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) // test functional dependency derived from keys in where condition tk.MustQuery("select * from t where c = d group by b, c") tk.MustQuery("select t.*, x.* from t, x where t.a = x.a group by t.a") @@ -511,7 +511,7 @@ func (s *testSuite) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select t.*, x.* from t, x where t.b = x.a group by t.b, t.d") tk.MustQuery("select t.b, x.* from t, x where t.b = x.a group by t.b") _, err = tk.Exec("select t.*, x.* from t, x where t.c = x.a group by t.b, t.c") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) // test functional dependency derived from keys in join tk.MustQuery("select t.*, x.* from t inner join x on t.a = x.a group by t.a") tk.MustQuery("select t.*, x.* from t inner join x on (t.b = x.b and t.d = x.d) group by t.b, x.d") @@ -521,9 +521,9 @@ func (s *testSuite) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select x.b, t.* from t right join x on x.b = t.b group by x.b, t.d") tk.MustQuery("select x.b, t.* from t right join x on t.b = x.b group by x.b, t.d") _, err = tk.Exec("select t.b, x.* from t right join x on t.b = x.b group by t.b, x.d") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select t.b, x.* from t right join x on t.b = x.b group by t.b, x.d") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) // FixMe: test functional dependency of derived table //tk.MustQuery("select * from (select * from t) as e group by a") @@ -534,10 +534,10 @@ func (s *testSuite) TestOnlyFullGroupBy(c *C) { // test order by tk.MustQuery("select c from t group by c,d order by d") _, err = tk.Exec("select c from t group by c order by d") - c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) // test ambiguous column _, err = tk.Exec("select c from t,x group by t.c") - c.Assert(terror.ErrorEqual(err, plan.ErrAmbiguous), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrAmbiguous), IsTrue, Commentf("err %v", err)) } func (s *testSuite) TestHaving(c *C) { diff --git a/executor/ddl_test.go b/executor/ddl_test.go index f91bcf6d3be8e..87c69c5b95ac2 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -411,11 +411,11 @@ func (s *testSuite) TestSetDDLReorgWorkerCnt(c *C) { tk.MustExec("set tidb_ddl_reorg_worker_cnt = 100") c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(100)) _, err := tk.Exec("set tidb_ddl_reorg_worker_cnt = invalid_val") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set tidb_ddl_reorg_worker_cnt = 100") c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(100)) _, err = tk.Exec("set tidb_ddl_reorg_worker_cnt = -1") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set tidb_ddl_reorg_worker_cnt = 100") res := tk.MustQuery("select @@tidb_ddl_reorg_worker_cnt") diff --git a/executor/executor_test.go b/executor/executor_test.go index 634c5badbd6ea..7a8343240bcdb 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -987,7 +987,7 @@ func (s *testSuite) TestUnion(c *C) { c.Assert(terr.Code(), Equals, terror.ErrCode(mysql.ErrWrongUsage)) _, err = tk.Exec("(select a from t order by a) union all select a from t limit 1 union all select a from t limit 1") - c.Assert(terror.ErrorEqual(err, plan.ErrWrongUsage), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrWrongUsage), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("(select a from t limit 1) union all select a from t limit 1") c.Assert(err, IsNil) @@ -1821,7 +1821,7 @@ func (s *testSuite) TestHistoryRead(c *C) { // Set snapshot to a time before save point will fail. _, err := tk.Exec("set @@tidb_snapshot = '2006-01-01 15:04:05.999999'") - c.Assert(terror.ErrorEqual(err, variable.ErrSnapshotTooOld), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrSnapshotTooOld), IsTrue, Commentf("err %v", err)) // SnapshotTS Is not updated if check failed. c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) @@ -1991,7 +1991,7 @@ func (s *testSuite) TestTiDBCurrentTS(c *C) { tk.MustQuery("select @@tidb_current_ts").Check(testkit.Rows("0")) _, err := tk.Exec("set @@tidb_current_ts = '1'") - c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue, Commentf("err %v", err)) } func (s *testSuite) TestSelectForUpdate(c *C) { @@ -2074,9 +2074,9 @@ func (s *testSuite) TestEmptyEnum(c *C) { tk.MustExec("create table t (e enum('Y', 'N'))") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") _, err := tk.Exec("insert into t values (0)") - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) + c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("insert into t values ('abc')") - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) + c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) tk.MustExec("set sql_mode=''") tk.MustExec("insert into t values (0)") diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 215b6eb97007f..c22bbcdb699aa 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -61,7 +61,7 @@ func (s *testSuite) TestPrepared(c *C) { // incorrect SQLs in prepare. issue #3738, SQL in prepare stmt is parsed in DoPrepare. _, err = tk.Exec(`prepare p from "delete from t where a = 7 or 1=1/*' and b = 'p'";`) - c.Assert(terror.ErrorEqual(err, errors.New(`[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '/*' and b = 'p'' at line 1`)), IsTrue) + c.Assert(terror.ErrorEqual(err, errors.New(`[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use near '/*' and b = 'p'' at line 1`)), IsTrue, Commentf("err %v", err)) // The `stmt_test5` should not be found. _, err = tk.Exec(`set @a = 1; execute stmt_test_5 using @a;`) diff --git a/executor/set_test.go b/executor/set_test.go index a2c7c6c2773e1..c02eabf9e466f 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -252,19 +252,19 @@ func (s *testSuite) TestValidateSetVar(c *C) { tk := testkit.NewTestKit(c, s.store) _, err := tk.Exec("set global tidb_distsql_scan_concurrency='fff';") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set global tidb_distsql_scan_concurrency=-1;") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set @@tidb_distsql_scan_concurrency='fff';") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set @@tidb_distsql_scan_concurrency=-1;") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set @@tidb_batch_delete='ok';") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set @@tidb_batch_delete='On';") tk.MustExec("set @@tidb_batch_delete='oFf';") @@ -272,10 +272,10 @@ func (s *testSuite) TestValidateSetVar(c *C) { tk.MustExec("set @@tidb_batch_delete=0;") _, err = tk.Exec("set @@tidb_batch_delete=3;") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set @@tidb_mem_quota_mergejoin='tidb';") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set @@group_concat_max_len=1") tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect group_concat_max_len value: '1'")) @@ -283,11 +283,11 @@ func (s *testSuite) TestValidateSetVar(c *C) { result.Check(testkit.Rows("4")) _, err = tk.Exec("set @@group_concat_max_len = 18446744073709551616") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) // Test illegal type _, err = tk.Exec("set @@group_concat_max_len='hello'") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set @@default_week_format=-1") tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect default_week_format value: '-1'")) @@ -300,10 +300,10 @@ func (s *testSuite) TestValidateSetVar(c *C) { result.Check(testkit.Rows("7")) _, err = tk.Exec("set @@error_count = 0") - c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("set @@warning_count = 0") - c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrReadOnly), IsTrue, Commentf("err %v", err)) tk.MustExec("set time_zone='SySTeM'") result = tk.MustQuery("select @@time_zone;") diff --git a/executor/simple_test.go b/executor/simple_test.go index bc2b9d505d091..208fcb79caaf2 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -172,7 +172,7 @@ func (s *testSuite) TestUser(c *C) { // Test 'identified by password' createUserSQL = `CREATE USER 'test1'@'localhost' identified by password 'xxx';` _, err = tk.Exec(createUserSQL) - c.Assert(terror.ErrorEqual(executor.ErrPasswordFormat, err), IsTrue) + c.Assert(terror.ErrorEqual(executor.ErrPasswordFormat, err), IsTrue, Commentf("err %v", err)) createUserSQL = `CREATE USER 'test1'@'localhost' identified by password '*3D56A309CD04FA2EEF181462E59011F075C89548';` tk.MustExec(createUserSQL) dropUserSQL = `DROP USER 'test1'@'localhost';` @@ -187,7 +187,7 @@ func (s *testSuite) TestUser(c *C) { // Test drop user meet error _, err = tk.Exec(dropUserSQL) - c.Assert(terror.ErrorEqual(err, executor.ErrCannotUser.GenByArgs("DROP USER", "")), IsTrue) + c.Assert(terror.ErrorEqual(err, executor.ErrCannotUser.GenByArgs("DROP USER", "")), IsTrue, Commentf("err %v", err)) createUserSQL = `CREATE USER 'test1'@'localhost'` tk.MustExec(createUserSQL) @@ -196,7 +196,7 @@ func (s *testSuite) TestUser(c *C) { dropUserSQL = `DROP USER 'test1'@'localhost', 'test2'@'localhost', 'test3'@'localhost';` _, err = tk.Exec(dropUserSQL) - c.Assert(terror.ErrorEqual(err, executor.ErrCannotUser.GenByArgs("DROP USER", "")), IsTrue) + c.Assert(terror.ErrorEqual(err, executor.ErrCannotUser.GenByArgs("DROP USER", "")), IsTrue, Commentf("err %v", err)) } func (s *testSuite) TestSetPwd(c *C) { @@ -223,7 +223,7 @@ func (s *testSuite) TestSetPwd(c *C) { ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd1", Hostname: "localhost"} // Session user doesn't exist. _, err = tk.Exec(setPwdSQL) - c.Check(terror.ErrorEqual(err, executor.ErrPasswordNoMatch), IsTrue) + c.Check(terror.ErrorEqual(err, executor.ErrPasswordNoMatch), IsTrue, Commentf("err %v", err)) // normal ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd", Hostname: "localhost"} tk.MustExec(setPwdSQL) diff --git a/executor/statement_context_test.go b/executor/statement_context_test.go index 4601a8ee2c979..5808488fed575 100644 --- a/executor/statement_context_test.go +++ b/executor/statement_context_test.go @@ -71,7 +71,7 @@ func (s *testSuite) TestStatementContext(c *C) { tk.MustExec(strictModeSQL) _, err = tk.Exec("insert sc2 values (unhex('4040ffff'))") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, table.ErrTruncateWrongValue), IsTrue) + c.Assert(terror.ErrorEqual(err, table.ErrTruncateWrongValue), IsTrue, Commentf("err %v", err)) tk.MustExec("set @@tidb_skip_utf8_check = '1'") _, err = tk.Exec("insert sc2 values (unhex('4040ffff'))") diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index 1c8bd20507229..ca8f63cb353ba 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -95,7 +95,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings := sc.GetWarnings() lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) originFlag := tp1.Flag tp1.Flag |= mysql.UnsignedFlag @@ -106,7 +106,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings = sc.GetWarnings() lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrCastNegIntAsUnsigned, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrCastNegIntAsUnsigned, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) tp1.Flag = originFlag previousWarnings := len(sc.GetWarnings()) @@ -125,7 +125,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings = sc.GetWarnings() lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) // cast('18446744073709551616' as signed); mask := ^mysql.UnsignedFlag @@ -137,7 +137,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings = sc.GetWarnings() lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) // cast('18446744073709551614' as signed); f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("18446744073709551614"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) @@ -147,7 +147,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings = sc.GetWarnings() lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrCastAsSignedOverflow, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrCastAsSignedOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) // create table t1(s1 time); // insert into t1 values('11:11:11'); @@ -169,7 +169,7 @@ func (s *testEvaluatorSuite) TestCast(c *C) { warnings = sc.GetWarnings() lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) sc = origSc // cast(bad_string as decimal) diff --git a/expression/builtin_encryption_test.go b/expression/builtin_encryption_test.go index 8b3ee9c2b5b5e..5d80199f15778 100644 --- a/expression/builtin_encryption_test.go +++ b/expression/builtin_encryption_test.go @@ -365,7 +365,7 @@ func (s *testEvaluatorSuite) TestPassword(c *C) { c.Assert(len(warnings), Equals, warnCount+1) lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(errDeprecatedSyntaxNoReplacement, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(errDeprecatedSyntaxNoReplacement, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) warnCount = len(warnings) } else { diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 7341f7f1055dc..8cb7fd923e719 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -786,7 +786,7 @@ func (s *testEvaluatorSuite) TestSpaceSig(c *C) { warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() c.Assert(len(warnings), Equals, 1) lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) } func (s *testEvaluatorSuite) TestLocate(c *C) { @@ -1350,7 +1350,7 @@ func (s *testEvaluatorSuite) TestRpadSig(c *C) { warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() c.Assert(len(warnings), Equals, 1) lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue) + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) } func (s *testEvaluatorSuite) TestInstr(c *C) { diff --git a/expression/integration_test.go b/expression/integration_test.go index 2bb7ba30c1f31..52ee004d0e1f4 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1078,15 +1078,15 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { tk.MustExec(`create table t(a bigint)`) _, err := tk.Exec(`insert into t select year("aa")`) c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue, Commentf("err %v", err)) _, err = tk.Exec(`insert into t select year("0000-00-00 00:00:00")`) c.Assert(err, NotNil) - c.Assert(types.ErrIncorrectDatetimeValue.Equal(err), IsTrue) + c.Assert(types.ErrIncorrectDatetimeValue.Equal(err), IsTrue, Commentf("err %v", err)) tk.MustExec(`insert into t select 1`) _, err = tk.Exec(`update t set a = year("aa")`) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue, Commentf("err %v", err)) _, err = tk.Exec(`delete from t where a = year("aa")`) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue, Commentf("err %v", err)) // for month result = tk.MustQuery(`select month("2013-01-09"), month("2013-00-09"), month("000-01-09"), month("1-01-09"), month("20131-01-09"), month(null);`) diff --git a/kv/union_store_test.go b/kv/union_store_test.go index c79c2207663a6..6241e853582de 100644 --- a/kv/union_store_test.go +++ b/kv/union_store_test.go @@ -119,7 +119,7 @@ func (s *testUnionStoreSuite) TestLazyConditionCheck(c *C) { s.us.SetOption(PresumeKeyNotExists, nil) s.us.SetOption(PresumeKeyNotExistsError, ErrNotExist) _, err = s.us.Get([]byte("2")) - c.Assert(terror.ErrorEqual(err, ErrNotExist), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrNotExist), IsTrue, Commentf("err %v", err)) err = s.us.CheckLazyConditionPairs() c.Assert(err, NotNil) diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index 507b455f05372..fdfce13a338fe 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -193,7 +193,7 @@ func (ts ConnTestSuite) TestParseStmtArgs(c *C) { } for _, tt := range tests { err := parseStmtArgs(tt.args.args, tt.args.boundParams, tt.args.nullBitmap, tt.args.paramTypes, tt.args.paramValues) - c.Assert(terror.ErrorEqual(err, tt.err), IsTrue) + c.Assert(terror.ErrorEqual(err, tt.err), IsTrue, Commentf("err %v", err)) c.Assert(tt.args.args[0], Equals, tt.expect) } } diff --git a/session/session_test.go b/session/session_test.go index 2179a89e1c7a2..05690a1e13e94 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -372,23 +372,23 @@ func (s *testSessionSuite) TestGetSysVariables(c *C) { tk.MustExec("select @@session.warning_count") tk.MustExec("select @@local.warning_count") _, err := tk.Exec("select @@global.warning_count") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) // Test ScopeGlobal tk.MustExec("select @@max_connections") tk.MustExec("select @@global.max_connections") _, err = tk.Exec("select @@session.max_connections") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select @@local.max_connections") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) // Test ScopeNone tk.MustExec("select @@performance_schema_max_mutex_classes") tk.MustExec("select @@global.performance_schema_max_mutex_classes") _, err = tk.Exec("select @@session.performance_schema_max_mutex_classes") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) _, err = tk.Exec("select @@local.performance_schema_max_mutex_classes") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) } func (s *testSessionSuite) TestRetryResetStmtCtx(c *C) { @@ -1259,12 +1259,12 @@ func (s *testSessionSuite) TestUnique(c *C) { _, err := tk.Exec("commit") c.Assert(err, NotNil) // Check error type and error message - c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) + c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue, Commentf("err %v", err)) c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'PRIMARY'") _, err = tk1.Exec("commit") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) + c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue, Commentf("err %v", err)) c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'val'") // Test for https://github.com/pingcap/tidb/issues/463 @@ -1596,7 +1596,7 @@ func (s *testSchemaSuite) TestCommitWhenSchemaChanged(c *C) { // When tk1 commit, it will find schema already changed. tk1.MustExec("insert into t values (4, 4)") _, err := tk1.Exec("commit") - c.Assert(terror.ErrorEqual(err, plan.ErrWrongValueCountOnRow), IsTrue) + c.Assert(terror.ErrorEqual(err, plan.ErrWrongValueCountOnRow), IsTrue, Commentf("err %v", err)) } func (s *testSchemaSuite) TestRetrySchemaChange(c *C) { @@ -2154,9 +2154,9 @@ func (s *testSessionSuite) TestSetGroupConcatMaxLen(c *C) { result.Check(testkit.Rows("4")) _, err := tk.Exec("set @@group_concat_max_len = 18446744073709551616") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) // Test illegal type _, err = tk.Exec("set @@group_concat_max_len='hello'") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index d52f7835dd6dd..5e8350e8dd46c 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -106,7 +106,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(err, IsNil) _, _, err = it.Next() - c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue) + c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue, Commentf("err %v", err)) it.Close() _, err = index.Create(mockCtx, txn, values, 0) @@ -127,14 +127,14 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(hit, IsFalse) _, _, err = it.Next() - c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue) + c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue, Commentf("err %v", err)) it.Close() it, err = index.SeekFirst(txn) c.Assert(err, IsNil) _, _, err = it.Next() - c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue) + c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue, Commentf("err %v", err)) it.Close() err = txn.Commit(context.Background()) diff --git a/types/convert_test.go b/types/convert_test.go index 659962177b314..b7ef6055a46ef 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -238,20 +238,20 @@ func (s *testTypeConvertSuite) TestConvertType(c *C) { c.Assert(err, IsNil, Commentf(errors.ErrorStack(err))) c.Assert(v.(*MyDecimal).String(), Equals, "3.1416") v, err = Convert("3.1415926", ft) - c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue, Commentf("err %v", err)) c.Assert(v.(*MyDecimal).String(), Equals, "3.1416") v, err = Convert("99999", ft) - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) c.Assert(v.(*MyDecimal).String(), Equals, "9999.9999") v, err = Convert("-10000", ft) - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) c.Assert(v.(*MyDecimal).String(), Equals, "-9999.9999") // Test Datum.ToDecimal with bad number. d := NewDatum("hello") sc := new(stmtctx.StatementContext) v, err = d.ToDecimal(sc) - c.Assert(terror.ErrorEqual(err, ErrBadNumber), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrBadNumber), IsTrue, Commentf("err %v", err)) sc.IgnoreTruncate = true v, err = d.ToDecimal(sc) @@ -288,7 +288,7 @@ func (s *testTypeConvertSuite) TestConvertType(c *C) { _, err = Convert("d", ft) c.Assert(err, NotNil) v, err = Convert(4, ft) - c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue, Commentf("err %v", err)) c.Assert(v, DeepEquals, Enum{}) ft = NewFieldType(mysql.TypeSet) @@ -343,7 +343,7 @@ func (s *testTypeConvertSuite) TestConvertToString(c *C) { ft.Flen = 10 ft.Decimal = 5 v, err := Convert(3.1415926, ft) - c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrTruncated), IsTrue, Commentf("err %v", err)) testToString(c, v, "3.14159") _, err = ToString(&invalidMockType{}) @@ -385,7 +385,7 @@ func testStrToInt(c *C, str string, expect int64, truncateAsErr bool, expectErr sc.IgnoreTruncate = !truncateAsErr val, err := StrToInt(sc, str) if expectErr != nil { - c.Assert(terror.ErrorEqual(err, expectErr), IsTrue) + c.Assert(terror.ErrorEqual(err, expectErr), IsTrue, Commentf("err %v", err)) } else { c.Assert(err, IsNil) c.Assert(val, Equals, expect) @@ -397,7 +397,7 @@ func testStrToUint(c *C, str string, expect uint64, truncateAsErr bool, expectEr sc.IgnoreTruncate = !truncateAsErr val, err := StrToUint(sc, str) if expectErr != nil { - c.Assert(terror.ErrorEqual(err, expectErr), IsTrue) + c.Assert(terror.ErrorEqual(err, expectErr), IsTrue, Commentf("err %v", err)) } else { c.Assert(err, IsNil) c.Assert(val, Equals, expect) @@ -409,7 +409,7 @@ func testStrToFloat(c *C, str string, expect float64, truncateAsErr bool, expect sc.IgnoreTruncate = !truncateAsErr val, err := StrToFloat(sc, str) if expectErr != nil { - c.Assert(terror.ErrorEqual(err, expectErr), IsTrue) + c.Assert(terror.ErrorEqual(err, expectErr), IsTrue, Commentf("err %v", err)) } else { c.Assert(err, IsNil) c.Assert(val, Equals, expect) @@ -685,9 +685,9 @@ func (s *testTypeConvertSuite) TestGetValidFloat(c *C) { c.Assert(err, IsNil) } _, err := floatStrToIntStr("1e9223372036854775807") - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) _, err = floatStrToIntStr("1e21") - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue) + c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) } // TestConvertTime tests time related conversion. diff --git a/types/etc_test.go b/types/etc_test.go index 92e7304787f29..4a5e161f2e32e 100644 --- a/types/etc_test.go +++ b/types/etc_test.go @@ -190,6 +190,6 @@ func (s *testTypeEtcSuite) TestTruncate(c *C) { for _, t := range tbl { f, err := TruncateFloat(t.Input, t.Flen, t.Decimal) c.Assert(f, Equals, t.Expect) - c.Assert(terror.ErrorEqual(err, t.Err), IsTrue) + c.Assert(terror.ErrorEqual(err, t.Err), IsTrue, Commentf("err %v", err)) } } diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index 645d2357dc703..667213a4e6524 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -746,9 +746,9 @@ func (s *testCodecSuite) TestDecimal(c *C) { d := types.NewDecFromStringForTest("-123.123456789") _, err := EncodeDecimal(nil, d, 20, 5) - c.Assert(terror.ErrorEqual(err, types.ErrTruncated), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrTruncated), IsTrue, Commentf("err %v", err)) _, err = EncodeDecimal(nil, d, 12, 10) - c.Assert(terror.ErrorEqual(err, types.ErrOverflow), IsTrue) + c.Assert(terror.ErrorEqual(err, types.ErrOverflow), IsTrue, Commentf("err %v", err)) sc.IgnoreTruncate = true decimalDatum := types.NewDatum(d) From 30e3e9e0ff876a3a08731e8681ad5df82ba62fcf Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Thu, 16 Aug 2018 23:54:12 +0800 Subject: [PATCH 69/87] *: set the proper customized timezone (#7372) --- sessionctx/variable/varsutil.go | 2 +- sessionctx/variable/varsutil_test.go | 4 +- store/mockstore/mocktikv/analyze.go | 10 +- store/mockstore/mocktikv/cop_handler_dag.go | 25 +++-- .../mocktikv/cop_handler_dag_test.go | 104 ++++++++++++++++++ 5 files changed, 129 insertions(+), 16 deletions(-) create mode 100644 store/mockstore/mocktikv/cop_handler_dag_test.go diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 014b129a942e8..492125fc09c50 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -398,7 +398,7 @@ func parseTimeZone(s string) (*time.Location, error) { if s[0] == '-' { ofst = -ofst } - return time.FixedZone("UTC", ofst), nil + return time.FixedZone("", ofst), nil } } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 9af93aee4d61e..2c2a96a25ea4c 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -128,8 +128,8 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { {"US/Eastern", "US/Eastern", true, 5 * time.Hour}, //TODO: Check it out and reopen this case. //{"SYSTEM", "Local", false, 0}, - {"+10:00", "UTC", true, -10 * time.Hour}, - {"-6:00", "UTC", true, 6 * time.Hour}, + {"+10:00", "", true, -10 * time.Hour}, + {"-6:00", "", true, 6 * time.Hour}, } for _, tt := range tests { err = SetSessionSystemVar(v, TimeZone, types.NewStringDatum(tt.input)) diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index 6615a626d2fde..4227a16081184 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -14,8 +14,6 @@ package mocktikv import ( - "time" - "github.com/golang/protobuf/proto" "github.com/juju/errors" "github.com/pingcap/kvproto/pkg/coprocessor" @@ -118,9 +116,13 @@ type analyzeColumnsExec struct { fields []*ast.ResultField } -func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeReq *tipb.AnalyzeReq) (*coprocessor.Response, error) { +func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeReq *tipb.AnalyzeReq) (_ *coprocessor.Response, err error) { sc := flagsToStatementContext(analyzeReq.Flags) - sc.TimeZone = time.FixedZone("UTC", int(analyzeReq.TimeZoneOffset)) + sc.TimeZone, err = constructTimeZone("", int(analyzeReq.TimeZoneOffset)) + if err != nil { + return nil, errors.Trace(err) + } + evalCtx := &evalContext{sc: sc} columns := analyzeReq.ColReq.ColumnsInfo evalCtx.setColumnInfo(columns) diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 0e40eb6211b26..861d274c7832c 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -145,17 +145,13 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex if err != nil { return nil, nil, nil, errors.Trace(err) } - sc := flagsToStatementContext(dagReq.Flags) - // retrieving timezone by name first. When name is set, it means we need - // consider daylight saving time. If it is not, we can use offset. - if dagReq.TimeZoneName != "" { - if sc.TimeZone, err = LocCache.getLoc(dagReq.TimeZoneName); err != nil { - return nil, nil, nil, errors.Trace(err) - } - } else { - sc.TimeZone = time.FixedZone("UTC", int(dagReq.TimeZoneOffset)) + sc := flagsToStatementContext(dagReq.Flags) + sc.TimeZone, err = constructTimeZone(dagReq.TimeZoneName, int(dagReq.TimeZoneOffset)) + if err != nil { + return nil, nil, nil, errors.Trace(err) } + ctx := &dagContext{ dagReq: dagReq, keyRanges: req.Ranges, @@ -168,6 +164,17 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex return ctx, e, dagReq, err } +// constructTimeZone constructs timezone by name first. When the timezone name +// is set, the daylight saving problem must be considered. Otherwise the +// timezone offset in seconds east of UTC is used to constructed the timezone. +func constructTimeZone(name string, offset int) (*time.Location, error) { + if name != "" { + return LocCache.getLoc(name) + } else { + return time.FixedZone("", offset), nil + } +} + func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Request) (tikvpb.Tikv_CoprocessorStreamClient, error) { dagCtx, e, dagReq, err := h.buildDAGExecutor(req) if err != nil { diff --git a/store/mockstore/mocktikv/cop_handler_dag_test.go b/store/mockstore/mocktikv/cop_handler_dag_test.go new file mode 100644 index 0000000000000..3e6f6acb27fc8 --- /dev/null +++ b/store/mockstore/mocktikv/cop_handler_dag_test.go @@ -0,0 +1,104 @@ +// Copyright 2018-present, 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 mocktikv + +import ( + "time" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testRPCHandlerSuite{}) + +type testRPCHandlerSuite struct { +} + +func (s *testRPCHandlerSuite) TestConstructTimezone(c *C) { + secondsEastOfUTC := int((8 * time.Hour).Seconds()) + loc, err := constructTimeZone("", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc := time.Date(2018, 8, 15, 20, 0, 0, 0, loc) + timeInUTC := time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + secondsEastOfUTC = int((-8 * time.Hour).Seconds()) + loc, err = constructTimeZone("", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 12, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 20, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + secondsEastOfUTC = 0 + loc, err = constructTimeZone("", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 20, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 20, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + secondsEastOfUTC = int((23 * time.Hour).Seconds()) + loc, err = constructTimeZone("UTC", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 12, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + secondsEastOfUTC = int((-23 * time.Hour).Seconds()) + loc, err = constructTimeZone("UTC", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 12, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + loc, err = constructTimeZone("UTC", 0) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 12, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + secondsEastOfUTC = int((-23 * time.Hour).Seconds()) + loc, err = constructTimeZone("Asia/Shanghai", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 20, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + secondsEastOfUTC = int((23 * time.Hour).Seconds()) + loc, err = constructTimeZone("Asia/Shanghai", secondsEastOfUTC) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 20, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the seconds east of UTC is ignored by the function + // constructTimeZone(). + loc, err = constructTimeZone("Asia/Shanghai", 0) + c.Assert(err, IsNil) + timeInLoc = time.Date(2018, 8, 15, 20, 0, 0, 0, loc) + timeInUTC = time.Date(2018, 8, 15, 12, 0, 0, 0, time.UTC) + c.Assert(timeInLoc.Equal(timeInUTC), IsTrue) + + // test the timezone name is not existed. + loc, err = constructTimeZone("asia/not-exist", 0) + c.Assert(err.Error(), Equals, "invalid name for timezone asia/not-exist") +} From 57f191408c75eba825425b32375381f461482c44 Mon Sep 17 00:00:00 2001 From: lysu Date: Fri, 17 Aug 2018 10:29:34 +0800 Subject: [PATCH 70/87] server: fix column info for resultType/flag for enum&set type. (#7417) --- server/column.go | 28 ++++++++++++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/server/column.go b/server/column.go index bf481253ab65f..27eee7fccc780 100644 --- a/server/column.go +++ b/server/column.go @@ -13,6 +13,10 @@ package server +import ( + "github.com/pingcap/tidb/mysql" +) + // ColumnInfo contains information of a column type ColumnInfo struct { Schema string @@ -42,8 +46,8 @@ func (column *ColumnInfo) Dump(buffer []byte) []byte { buffer = dumpUint16(buffer, column.Charset) buffer = dumpUint32(buffer, column.ColumnLength) - buffer = append(buffer, column.Type) - buffer = dumpUint16(buffer, column.Flag) + buffer = append(buffer, dumpType(column.Type)) + buffer = dumpUint16(buffer, dumpFlag(column.Type, column.Flag)) buffer = append(buffer, column.Decimal) buffer = append(buffer, 0, 0) @@ -54,3 +58,23 @@ func (column *ColumnInfo) Dump(buffer []byte) []byte { return buffer } + +func dumpFlag(tp byte, flag uint16) uint16 { + switch tp { + case mysql.TypeSet: + return flag | uint16(mysql.SetFlag) + case mysql.TypeEnum: + return flag | uint16(mysql.TypeEnum) + default: + return flag + } +} + +func dumpType(tp byte) byte { + switch tp { + case mysql.TypeSet, mysql.TypeEnum: + return mysql.TypeString + default: + return tp + } +} From 2ae4f7f3d2b1105f66571ff79e9cd5c395959c2f Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Fri, 17 Aug 2018 13:52:28 +0800 Subject: [PATCH 71/87] parser: add 'IDENTIFIED WITH' option on create user statement (#7402) * fix issue 7295 --- parser/parser.y | 17 +++++++++++++++++ parser/parser_test.go | 3 +++ 2 files changed, 20 insertions(+) diff --git a/parser/parser.y b/parser/parser.y index 73bc66d76c556..fca1d0fb1ae2b 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -6561,6 +6561,23 @@ AuthOption: ByAuthString: true, } } +| "IDENTIFIED" "WITH" StringName + { + $$ = nil + } +| "IDENTIFIED" "WITH" StringName "BY" AuthString + { + $$ = &ast.AuthOption { + AuthString: $5.(string), + ByAuthString: true, + } + } +| "IDENTIFIED" "WITH" StringName "AS" HashString + { + $$ = &ast.AuthOption{ + HashString: $5.(string), + } + } | "IDENTIFIED" "BY" "PASSWORD" HashString { $$ = &ast.AuthOption{ diff --git a/parser/parser_test.go b/parser/parser_test.go index b38e1e2de34b0..05f4934391dd2 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1852,6 +1852,9 @@ func (s *testParserSuite) TestPrivilege(c *C) { {"CREATE USER 'uesr1'@'localhost'", true}, {"CREATE USER 'uesr1'@`localhost`", true}, {"CREATE USER `uesr1`@'localhost'", true}, + {"create user 'bug19354014user'@'%' identified WITH mysql_native_password", true}, + {"create user 'bug19354014user'@'%' identified WITH mysql_native_password by 'new-password'", true}, + {"create user 'bug19354014user'@'%' identified WITH mysql_native_password as 'hashstring'", true}, {`CREATE USER IF NOT EXISTS 'root'@'localhost' IDENTIFIED BY 'new-password'`, true}, {`CREATE USER 'root'@'localhost' IDENTIFIED BY 'new-password'`, true}, {`CREATE USER 'root'@'localhost' IDENTIFIED BY PASSWORD 'hashstring'`, true}, From 8199f60492f4c0c37d870fba447959bed0c3edb2 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Fri, 17 Aug 2018 01:14:19 -0500 Subject: [PATCH 72/87] ddl:support drop index for the partitioned table. (#7306) --- ddl/db_test.go | 83 ++++++++++++++++++++++++++++++++++++++++++++- ddl/delete_range.go | 20 ++++++++--- ddl/index.go | 2 +- 3 files changed, 98 insertions(+), 7 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index edcfa89b8f96a..82750f446beda 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3122,10 +3122,91 @@ func (s *testDBSuite) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { s.testErrorCode(c, sql8, tmysql.ErrUniqueKeyNeedAllFieldsInPf) } +func (s *testDBSuite) TestPartitionDropIndex(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + done := make(chan error, 1) + s.tk.MustExec("use " + s.schemaName) + s.tk.MustExec("set @@session.tidb_enable_table_partition=1;") + s.tk.MustExec("drop table if exists partition_drop_idx;") + s.tk.MustExec(`create table partition_drop_idx ( + c1 int, c2 int, c3 int + ) + partition by range( c1 ) ( + partition p0 values less than (3), + partition p1 values less than (5), + partition p2 values less than (7), + partition p3 values less than (11), + partition p4 values less than (15), + partition p5 values less than (20), + partition p6 values less than (maxvalue) + );`) + + num := 20 + for i := 0; i < num; i++ { + s.mustExec(c, "insert into partition_drop_idx values (?, ?, ?)", i, i, i) + } + s.tk.MustExec("alter table partition_drop_idx add index idx1 (c1)") + + ctx := s.tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + t, err := is.TableByName(model.NewCIStr(s.schemaName), model.NewCIStr("partition_drop_idx")) + c.Assert(err, IsNil) + + var idx1 table.Index + for _, pidx := range t.Indices() { + if pidx.Meta().Name.L == "idx1" { + idx1 = pidx + break + } + } + c.Assert(idx1, NotNil) + + sessionExecInGoroutine(c, s.store, "drop index idx1 on partition_drop_idx;", done) + ticker := time.NewTicker(s.lease / 2) + defer ticker.Stop() +LOOP: + for { + select { + case err := <-done: + if err == nil { + break LOOP + } + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) + case <-ticker.C: + step := 10 + rand.Seed(time.Now().Unix()) + for i := num; i < num+step; i++ { + n := rand.Intn(num) + s.mustExec(c, "update partition_drop_idx set c2 = 1 where c1 = ?", n) + s.mustExec(c, "insert into partition_drop_idx values (?, ?, ?)", i, i, i) + } + num += step + } + } + + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr(s.schemaName), model.NewCIStr("partition_drop_idx")) + c.Assert(err, IsNil) + // Only one partition id test is taken here. + pid := t.Meta().Partition.Definitions[0].ID + var idxn table.Index + t.Indices() + for _, idx := range t.Indices() { + if idx.Meta().Name.L == "idx1" { + idxn = idx + break + } + } + c.Assert(idxn, IsNil) + idx := tables.NewIndex(pid, t.Meta(), idx1.Meta()) + checkDelRangeDone(c, ctx, idx) + s.tk.MustExec("drop table partition_drop_idx;") +} + func (s *testDBSuite) TestPartitionAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_table_partition = 1") + tk.MustExec("set @@session.tidb_enable_table_partition=1;") tk.MustExec(`create table partition_add_idx ( id int not null, hired date not null diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 648168ee15afe..2ad93e5928038 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -253,7 +253,6 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error } return nil } - startKey = tablecodec.EncodeTablePrefix(tableID) endKey := tablecodec.EncodeTablePrefix(tableID + 1) return doInsert(s, job.ID, tableID, startKey, endKey, now) @@ -279,12 +278,23 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error tableID := job.TableID var indexName interface{} var indexID int64 - if err := job.DecodeArgs(&indexName, &indexID); err != nil { + var partitionIDs []int64 + if err := job.DecodeArgs(&indexName, &indexID, &partitionIDs); err != nil { return errors.Trace(err) } - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) + } + } + } else { + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) + } } return nil } diff --git a/ddl/index.go b/ddl/index.go index 7e9e3b7c5be62..54a68511c1e54 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -436,7 +436,7 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.Args[0] = indexInfo.ID } else { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexInfo.ID) + job.Args = append(job.Args, indexInfo.ID, getPartitionIDs(tblInfo)) } default: err = ErrInvalidTableState.Gen("invalid table state %v", tblInfo.State) From 8f1b494055505a4934851b4da172a4a536b07903 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Fri, 17 Aug 2018 14:40:49 +0800 Subject: [PATCH 73/87] expression: fix missing microsecond for timestamp (#7418) --- expression/builtin_time.go | 4 ++++ expression/builtin_time_test.go | 9 +++++++++ 2 files changed, 13 insertions(+) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index e901bed01afd3..7d44e77746a80 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -2616,6 +2616,8 @@ func (du *baseDateArithmitical) add(ctx sessionctx.Context, date types.Time, int if goTime.Nanosecond() == 0 { date.Fsp = 0 + } else { + date.Fsp = 6 } date.Time = types.FromGoTime(goTime) @@ -2639,6 +2641,8 @@ func (du *baseDateArithmitical) sub(ctx sessionctx.Context, date types.Time, int if goTime.Nanosecond() == 0 { date.Fsp = 0 + } else { + date.Fsp = 6 } date.Time = types.FromGoTime(goTime) diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 223f2909595ef..1e7c472f2641c 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -758,6 +758,7 @@ func (s *testEvaluatorSuite) TestAddTimeSig(c *C) { {"2017-12-31 23:59:59", "00:00:01", "2018-01-01 00:00:00"}, {"2017-12-31 23:59:59", "1", "2018-01-01 00:00:00"}, {"2007-12-31 23:59:59.999999", "2 1:1:1.000002", "2008-01-03 01:01:01.000001"}, + {"2018-08-16 20:21:01", "00:00:00.000001", "2018-08-16 20:21:01.000001"}, } fc := funcs[ast.AddTime] for _, t := range tbl { @@ -771,6 +772,14 @@ func (s *testEvaluatorSuite) TestAddTimeSig(c *C) { c.Assert(result, Equals, t.expect) } + // This is a test for issue 7334 + du := newDateArighmeticalUtil() + now, _, err := evalNowWithFsp(s.ctx, 0) + c.Assert(err, IsNil) + res, _, err := du.add(s.ctx, now, "1", "MICROSECOND") + c.Assert(err, IsNil) + c.Assert(res.Fsp, Equals, 6) + tbl = []struct { Input string InputDuration string From 1558a9c096e77236e5c5b7cab125e1618c6272d8 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 17 Aug 2018 16:52:52 +0800 Subject: [PATCH 74/87] executor: fix duplicate row check when chunk size is small (#7393) --- executor/insert.go | 37 +++---------------------------------- executor/insert_common.go | 29 ++++++++++++++++++++++++----- executor/write_test.go | 1 + 3 files changed, 28 insertions(+), 39 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index c497b7bac2d4e..c69d4fafec8a9 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -28,26 +28,18 @@ import ( // InsertExec represents an insert executor. type InsertExec struct { *InsertValues - - batchInsertRowCount uint64 - OnDuplicate []*expression.Assignment - - Priority mysql.PriorityEnum - - finished bool + OnDuplicate []*expression.Assignment + Priority mysql.PriorityEnum + finished bool } func (e *InsertExec) insertOneRow(row []types.Datum) (int64, error) { - if err := e.checkBatchLimit(); err != nil { - return 0, errors.Trace(err) - } e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) h, err := e.Table.AddRecord(e.ctx, row, false) e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) if err != nil { return 0, errors.Trace(err) } - e.batchInsertRowCount++ return h, nil } @@ -92,25 +84,6 @@ func (e *InsertExec) exec(rows [][]types.Datum) error { return nil } -// checkBatchLimit check the batchSize limitation. -func (e *InsertExec) checkBatchLimit() error { - sessVars := e.ctx.GetSessionVars() - batchInsert := sessVars.BatchInsert && !sessVars.InTxn() - batchSize := sessVars.DMLBatchSize - if batchInsert && e.batchInsertRowCount >= uint64(batchSize) { - e.ctx.StmtCommit() - if err := e.ctx.NewTxn(); err != nil { - // We should return a special error for batch insert. - return ErrBatchInsertFail.Gen("BatchInsert failed with error: %v", err) - } - e.batchInsertRowCount = 0 - if !sessVars.LightningMode { - sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(e.ctx.Txn(), kv.TempTxnMemBufCap) - } - } - return nil -} - // batchUpdateDupRows updates multi-rows in batch if they are duplicate with rows in table. func (e *InsertExec) batchUpdateDupRows(newRows [][]types.Datum) error { err := e.batchGetInsertKeys(e.ctx, e.Table, newRows) @@ -249,10 +222,6 @@ func (e *InsertExec) doDupRowUpdate(handle int64, oldRow []types.Datum, newRow [ if err != nil { return nil, false, 0, errors.Trace(err) } - e.batchInsertRowCount++ - if err := e.checkBatchLimit(); err != nil { - return nil, false, 0, errors.Trace(err) - } if lastInsertID != 0 { e.lastInsertID = lastInsertID } diff --git a/executor/insert_common.go b/executor/insert_common.go index 99b4ca6d1f50e..7859cc68da855 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -19,6 +19,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/table" @@ -252,6 +253,11 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C chk := selectExec.newChunk() iter := chunk.NewIterator4Chunk(chk) rows := make([][]types.Datum, 0, e.ctx.GetSessionVars().MaxChunkSize) + + sessVars := e.ctx.GetSessionVars() + batchInsert := sessVars.BatchInsert && !sessVars.InTxn() + batchSize := sessVars.DMLBatchSize + for { err := selectExec.Next(ctx, chk) if err != nil { @@ -262,18 +268,31 @@ func (e *InsertValues) insertRowsFromSelect(ctx context.Context, cols []*table.C } for innerChunkRow := iter.Begin(); innerChunkRow != iter.End(); innerChunkRow = iter.Next() { - innerRow := innerChunkRow.GetDatumRow(fields) + innerRow := types.CopyRow(innerChunkRow.GetDatumRow(fields)) e.rowCount++ row, err := e.fillRowData(cols, innerRow) if err != nil { return errors.Trace(err) } rows = append(rows, row) + if batchInsert && e.rowCount%uint64(batchSize) == 0 { + if err := exec(rows); err != nil { + return errors.Trace(err) + } + e.ctx.StmtCommit() + rows = rows[:0] + if err := e.ctx.NewTxn(); err != nil { + // We should return a special error for batch insert. + return ErrBatchInsertFail.Gen("BatchInsert failed with error: %v", err) + } + if !sessVars.LightningMode { + sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(e.ctx.Txn(), kv.TempTxnMemBufCap) + } + } } - if err := exec(rows); err != nil { - return errors.Trace(err) - } - rows = rows[:0] + } + if err := exec(rows); err != nil { + return errors.Trace(err) } return nil } diff --git a/executor/write_test.go b/executor/write_test.go index aa8a7649b12e1..9d97a4ba5d7d0 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -586,6 +586,7 @@ commit;` create table m (id int primary key auto_increment, code int unique); insert tmp (code) values (1); insert tmp (code) values (1); + set tidb_max_chunk_size=1; insert m (code) select code from tmp on duplicate key update code = values(code);` tk.MustExec(testSQL) testSQL = `select * from m;` From 0f7bc02700d827dd101f09f6fd32c687365a573d Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 17 Aug 2018 18:31:32 +0800 Subject: [PATCH 75/87] docs: remove not worked MVCC api (#7400) --- docs/tidb_http_api.md | 6 ------ server/http_handler.go | 18 ++++++------------ server/http_status.go | 1 - 3 files changed, 6 insertions(+), 19 deletions(-) diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index 819c7c1933c0a..a75b5d39a1e15 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -70,12 +70,6 @@ curl http://{TiDBIP}:10080/mvcc/txn/{startTS}/{db}/{table} ``` -1. Get MVCC Information of the key with a specified handle ID - - ```shell - curl http://{TiDBIP}:10080/mvcc/txn/{startTS} - ``` - 1. Get MVCC Information by a hex value ```shell diff --git a/server/http_handler.go b/server/http_handler.go index e3b2551968a60..47ca85f63de4a 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -1243,18 +1243,12 @@ func (h *mvccTxnHandler) handleMvccGetByTxn(params map[string]string) (interface if err != nil { return nil, errors.Trace(err) } - - startKey := []byte("") - endKey := []byte("") - dbName := params[pDBName] - if len(dbName) > 0 { - tableID, err := h.getTableID(params[pDBName], params[pTableName]) - if err != nil { - return nil, errors.Trace(err) - } - startKey = tablecodec.EncodeTablePrefix(tableID) - endKey = tablecodec.EncodeRowKeyWithHandle(tableID, math.MaxInt64) + tableID, err := h.getTableID(params[pDBName], params[pTableName]) + if err != nil { + return nil, errors.Trace(err) } + startKey := tablecodec.EncodeTablePrefix(tableID) + endKey := tablecodec.EncodeRowKeyWithHandle(tableID, math.MaxInt64) return h.getMvccByStartTs(uint64(startTS), startKey, endKey) } @@ -1311,7 +1305,7 @@ func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request return } allVersionsMap := map[domain.ServerVersionInfo]struct{}{} - allVersions := []domain.ServerVersionInfo{} + var allVersions []domain.ServerVersionInfo for _, v := range allServersInfo { if _, ok := allVersionsMap[v.ServerVersionInfo]; ok { continue diff --git a/server/http_status.go b/server/http_status.go index 357594d3cb695..934769b881388 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -66,7 +66,6 @@ func (s *Server) startHTTPServer() { router.Handle("/regions/{regionID}", regionHandler{tikvHandlerTool}) router.Handle("/mvcc/key/{db}/{table}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByKey}) router.Handle("/mvcc/txn/{startTS}/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByTxn}) - router.Handle("/mvcc/txn/{startTS}", mvccTxnHandler{tikvHandlerTool, opMvccGetByTxn}) router.Handle("/mvcc/hex/{hexKey}", mvccTxnHandler{tikvHandlerTool, opMvccGetByHex}) router.Handle("/mvcc/index/{db}/{table}/{index}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) } From 4624785b377784ce8b63b596c55f95fa01f26cce Mon Sep 17 00:00:00 2001 From: mengnan <40080929+supernan1994@users.noreply.github.com> Date: Fri, 17 Aug 2018 23:12:37 +0800 Subject: [PATCH 76/87] expression: handle max_allowed_packet warnings for from_base64 function. (#7409) --- expression/builtin_string.go | 33 +++++++++++++++++++- expression/builtin_string_test.go | 52 +++++++++++++++++++++++++++++++ 2 files changed, 84 insertions(+), 1 deletion(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 100e78a2ac6d7..8b2a772565420 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -2959,18 +2959,39 @@ func (c *fromBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Exp } bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) bf.tp.Flen = mysql.MaxBlobWidth + + valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) + maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + types.SetBinChsClnFlag(bf.tp) - sig := &builtinFromBase64Sig{bf} + sig := &builtinFromBase64Sig{bf, maxAllowedPacket} return sig, nil } +// base64NeededDecodedLength return the base64 decoded string length. +func base64NeededDecodedLength(n int) int { + // Returns -1 indicate the result will overflow. + if strconv.IntSize == 64 && n > math.MaxInt64/3 { + return -1 + } + if strconv.IntSize == 32 && n > math.MaxInt32/3 { + return -1 + } + return n * 3 / 4 +} + type builtinFromBase64Sig struct { baseBuiltinFunc + maxAllowedPacket uint64 } func (b *builtinFromBase64Sig) Clone() builtinFunc { newSig := &builtinFromBase64Sig{} newSig.cloneFrom(&b.baseBuiltinFunc) + newSig.maxAllowedPacket = b.maxAllowedPacket return newSig } @@ -2981,6 +3002,16 @@ func (b *builtinFromBase64Sig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", true, errors.Trace(err) } + + needDecodeLen := base64NeededDecodedLength(len(str)) + if needDecodeLen == -1 { + return "", true, nil + } + if needDecodeLen > int(b.maxAllowedPacket) { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnAllowedPacketOverflowed.GenByArgs("from_base64", b.maxAllowedPacket)) + return "", true, nil + } + str = strings.Replace(str, "\t", "", -1) str = strings.Replace(str, " ", "", -1) result, err := base64.StdEncoding.DecodeString(str) diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 8cb7fd923e719..2f7b9d4e0ab93 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -1609,6 +1609,58 @@ func (s *testEvaluatorSuite) TestFromBase64(c *C) { } } +func (s *testEvaluatorSuite) TestFromBase64Sig(c *C) { + colTypes := []*types.FieldType{ + {Tp: mysql.TypeVarchar}, + } + + tests := []struct { + args string + expect string + isNil bool + maxAllowPacket uint64 + }{ + {string("YWJj"), string("abc"), false, 3}, + {string("YWJj"), "", true, 2}, + { + string("QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), + string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + false, + 70, + }, + { + string("QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), + "", + true, + 69, + }, + } + + args := []Expression{ + &Column{Index: 0, RetType: colTypes[0]}, + } + + for _, test := range tests { + resultType := &types.FieldType{Tp: mysql.TypeVarchar, Flen: mysql.MaxBlobWidth} + base := baseBuiltinFunc{args: args, ctx: s.ctx, tp: resultType} + fromBase64 := &builtinFromBase64Sig{base, test.maxAllowPacket} + + input := chunk.NewChunkWithCapacity(colTypes, 1) + input.AppendString(0, test.args) + res, isNull, err := fromBase64.evalString(input.GetRow(0)) + c.Assert(err, IsNil) + c.Assert(isNull, Equals, test.isNil) + if isNull { + warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, 1) + lastWarn := warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue) + s.ctx.GetSessionVars().StmtCtx.SetWarnings([]stmtctx.SQLWarn{}) + } + c.Assert(res, Equals, test.expect) + } +} + func (s *testEvaluatorSuite) TestInsert(c *C) { tests := []struct { args []interface{} From 2fbd5e7a704e1b0bb813de70f7a6b5a4120a2c07 Mon Sep 17 00:00:00 2001 From: H Date: Fri, 17 Aug 2018 23:39:47 +0800 Subject: [PATCH 77/87] expression: handle max_allowed_packet warnings for repeat function. (#7181) --- expression/builtin_string.go | 17 ++++++++++-- expression/builtin_string_test.go | 45 +++++++++++++++++++++++++++++++ 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 8b2a772565420..f63ea4fe483e7 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -555,17 +555,24 @@ func (c *repeatFunctionClass) getFunction(ctx sessionctx.Context, args []Express bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt) bf.tp.Flen = mysql.MaxBlobWidth SetBinFlagOrBinStr(args[0].GetType(), bf.tp) - sig := &builtinRepeatSig{bf} + valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) + maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + sig := &builtinRepeatSig{bf, maxAllowedPacket} return sig, nil } type builtinRepeatSig struct { baseBuiltinFunc + maxAllowedPacket uint64 } func (b *builtinRepeatSig) Clone() builtinFunc { newSig := &builtinRepeatSig{} newSig.cloneFrom(&b.baseBuiltinFunc) + newSig.maxAllowedPacket = b.maxAllowedPacket return newSig } @@ -576,6 +583,7 @@ func (b *builtinRepeatSig) evalString(row chunk.Row) (d string, isNull bool, err if isNull || err != nil { return "", isNull, errors.Trace(err) } + byteLength := len(str) num, isNull, err := b.args[1].EvalInt(b.ctx, row) if isNull || err != nil { @@ -588,7 +596,12 @@ func (b *builtinRepeatSig) evalString(row chunk.Row) (d string, isNull bool, err num = math.MaxInt32 } - if int64(len(str)) > int64(b.tp.Flen)/num { + if uint64(byteLength)*uint64(num) > b.maxAllowedPacket { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnAllowedPacketOverflowed.GenByArgs("repeat", b.maxAllowedPacket)) + return "", true, nil + } + + if int64(byteLength) > int64(b.tp.Flen)/num { return "", true, nil } return strings.Repeat(str, int(num)), false, nil diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 2f7b9d4e0ab93..09e6d961e4fd1 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -399,6 +399,51 @@ func (s *testEvaluatorSuite) TestRepeat(c *C) { c.Assert(v.GetString(), Equals, "") } +func (s *testEvaluatorSuite) TestRepeatSig(c *C) { + colTypes := []*types.FieldType{ + {Tp: mysql.TypeVarchar}, + {Tp: mysql.TypeLonglong}, + } + resultType := &types.FieldType{Tp: mysql.TypeVarchar, Flen: 1000} + args := []Expression{ + &Column{Index: 0, RetType: colTypes[0]}, + &Column{Index: 1, RetType: colTypes[1]}, + } + base := baseBuiltinFunc{args: args, ctx: s.ctx, tp: resultType} + repeat := &builtinRepeatSig{base, 1000} + + cases := []struct { + args []interface{} + warning int + res string + }{ + {[]interface{}{"a", int64(6)}, 0, "aaaaaa"}, + {[]interface{}{"a", int64(10001)}, 1, ""}, + {[]interface{}{"毅", int64(6)}, 0, "毅毅毅毅毅毅"}, + {[]interface{}{"毅", int64(334)}, 2, ""}, + } + + for _, t := range cases { + input := chunk.NewChunkWithCapacity(colTypes, 10) + input.AppendString(0, t.args[0].(string)) + input.AppendInt64(1, t.args[1].(int64)) + + res, isNull, err := repeat.evalString(input.GetRow(0)) + c.Assert(res, Equals, t.res) + c.Assert(err, IsNil) + if t.warning == 0 { + c.Assert(isNull, IsFalse) + } else { + c.Assert(isNull, IsTrue) + c.Assert(err, IsNil) + warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, t.warning) + lastWarn := warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue) + } + } +} + func (s *testEvaluatorSuite) TestLower(c *C) { defer testleak.AfterTest(c)() cases := []struct { From d5ab70cfdf883f139138a6fc42bc40c4ee34527e Mon Sep 17 00:00:00 2001 From: Du Chuan Date: Sun, 19 Aug 2018 14:55:15 +0800 Subject: [PATCH 78/87] executor,sessionctx: Add correctness for more system variables (#7196) --- executor/set_test.go | 136 ++++++++++++++++++++++++++ sessionctx/variable/sysvar.go | 20 +++- sessionctx/variable/varsutil.go | 165 +++++++++++++------------------- 3 files changed, 215 insertions(+), 106 deletions(-) diff --git a/executor/set_test.go b/executor/set_test.go index c02eabf9e466f..a4a5406c5faf0 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -309,4 +309,140 @@ func (s *testSuite) TestValidateSetVar(c *C) { result = tk.MustQuery("select @@time_zone;") result.Check(testkit.Rows("SYSTEM")) + // The following cases test value out of range and illegal type when setting system variables. + // See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html for more details. + tk.MustExec("set @@global.max_connections=100001") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '100001'")) + result = tk.MustQuery("select @@global.max_connections;") + result.Check(testkit.Rows("100000")) + + tk.MustExec("set @@global.max_connections=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) + result = tk.MustQuery("select @@global.max_connections;") + result.Check(testkit.Rows("1")) + + _, err = tk.Exec("set @@global.max_connections='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@global.max_connect_errors=18446744073709551615") + + tk.MustExec("set @@global.max_connect_errors=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connect_errors value: '-1'")) + result = tk.MustQuery("select @@global.max_connect_errors;") + result.Check(testkit.Rows("1")) + + _, err = tk.Exec("set @@global.max_connect_errors=18446744073709551616") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@global.max_connections=100001") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '100001'")) + result = tk.MustQuery("select @@global.max_connections;") + result.Check(testkit.Rows("100000")) + + tk.MustExec("set @@global.max_connections=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) + result = tk.MustQuery("select @@global.max_connections;") + result.Check(testkit.Rows("1")) + + _, err = tk.Exec("set @@global.max_connections='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@max_sort_length=1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_sort_length value: '1'")) + result = tk.MustQuery("select @@max_sort_length;") + result.Check(testkit.Rows("4")) + + tk.MustExec("set @@max_sort_length=-100") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_sort_length value: '-100'")) + result = tk.MustQuery("select @@max_sort_length;") + result.Check(testkit.Rows("4")) + + tk.MustExec("set @@max_sort_length=8388609") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect max_sort_length value: '8388609'")) + result = tk.MustQuery("select @@max_sort_length;") + result.Check(testkit.Rows("8388608")) + + _, err = tk.Exec("set @@max_sort_length='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@global.table_definition_cache=399") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect table_definition_cache value: '399'")) + result = tk.MustQuery("select @@global.table_definition_cache;") + result.Check(testkit.Rows("400")) + + tk.MustExec("set @@global.table_definition_cache=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect table_definition_cache value: '-1'")) + result = tk.MustQuery("select @@global.table_definition_cache;") + result.Check(testkit.Rows("400")) + + tk.MustExec("set @@global.table_definition_cache=524289") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect table_definition_cache value: '524289'")) + result = tk.MustQuery("select @@global.table_definition_cache;") + result.Check(testkit.Rows("524288")) + + _, err = tk.Exec("set @@global.table_definition_cache='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@old_passwords=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect old_passwords value: '-1'")) + result = tk.MustQuery("select @@old_passwords;") + result.Check(testkit.Rows("0")) + + tk.MustExec("set @@old_passwords=3") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect old_passwords value: '3'")) + result = tk.MustQuery("select @@old_passwords;") + result.Check(testkit.Rows("2")) + + _, err = tk.Exec("set @@old_passwords='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@tmp_table_size=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tmp_table_size value: '-1'")) + result = tk.MustQuery("select @@tmp_table_size;") + result.Check(testkit.Rows("1024")) + + tk.MustExec("set @@tmp_table_size=1020") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect tmp_table_size value: '1020'")) + result = tk.MustQuery("select @@tmp_table_size;") + result.Check(testkit.Rows("1024")) + + tk.MustExec("set @@tmp_table_size=167772161") + result = tk.MustQuery("select @@tmp_table_size;") + result.Check(testkit.Rows("167772161")) + + tk.MustExec("set @@tmp_table_size=18446744073709551615") + result = tk.MustQuery("select @@tmp_table_size;") + result.Check(testkit.Rows("18446744073709551615")) + + _, err = tk.Exec("set @@tmp_table_size=18446744073709551616") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + _, err = tk.Exec("set @@tmp_table_size='hello'") + c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) + + tk.MustExec("set @@global.connect_timeout=1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect connect_timeout value: '1'")) + result = tk.MustQuery("select @@global.connect_timeout;") + result.Check(testkit.Rows("2")) + + tk.MustExec("set @@global.connect_timeout=31536000") + result = tk.MustQuery("select @@global.connect_timeout;") + result.Check(testkit.Rows("31536000")) + + tk.MustExec("set @@global.connect_timeout=31536001") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect connect_timeout value: '31536001'")) + result = tk.MustQuery("select @@global.connect_timeout;") + result.Check(testkit.Rows("31536000")) + + result = tk.MustQuery("select @@sql_select_limit;") + result.Check(testkit.Rows("18446744073709551615")) + tk.MustExec("set @@sql_select_limit=default") + result = tk.MustQuery("select @@sql_select_limit;") + result.Check(testkit.Rows("18446744073709551615")) + + tk.MustExec("set @@global.flush_time=31536001") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect flush_time value: '31536001'")) + + tk.MustExec("set @@global.interactive_timeout=31536001") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect interactive_timeout value: '31536001'")) } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cd57cdc78f380..d62b2bea3ebe8 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -128,7 +128,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, "rand_seed2", ""}, {ScopeGlobal, "validate_password_number_count", "1"}, {ScopeSession, "gtid_next", ""}, - {ScopeGlobal | ScopeSession, "sql_select_limit", "18446744073709551615"}, + {ScopeGlobal | ScopeSession, SQLSelectLimit, "18446744073709551615"}, {ScopeGlobal, "ndb_show_foreign_key_mock_tables", ""}, {ScopeNone, "multi_range_count", "256"}, {ScopeGlobal | ScopeSession, DefaultWeekFormat, "0"}, @@ -136,7 +136,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "slave_transaction_retries", "10"}, {ScopeGlobal | ScopeSession, "default_storage_engine", "InnoDB"}, {ScopeNone, "ft_query_expansion_limit", "20"}, - {ScopeGlobal, "max_connect_errors", "100"}, + {ScopeGlobal, MaxConnectErrors, "100"}, {ScopeGlobal, "sync_binlog", "0"}, {ScopeNone, "max_digest_length", "1024"}, {ScopeNone, "innodb_force_load_corrupted", "OFF"}, @@ -146,7 +146,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "log_backward_compatible_user_definitions", ""}, {ScopeNone, "lc_messages_dir", "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, {ScopeGlobal, "ft_boolean_syntax", "+ -><()~*:\"\"&|"}, - {ScopeGlobal, "table_definition_cache", "1400"}, + {ScopeGlobal, TableDefinitionCache, "-1"}, {ScopeNone, SkipNameResolve, "0"}, {ScopeNone, "performance_schema_max_file_handles", "32768"}, {ScopeSession, "transaction_allow_batching", ""}, @@ -154,7 +154,7 @@ var defaultSysVars = []*SysVar{ {ScopeNone, "performance_schema_max_statement_classes", "168"}, {ScopeGlobal, "server_id", "0"}, {ScopeGlobal, "innodb_flushing_avg_loops", "30"}, - {ScopeGlobal | ScopeSession, "tmp_table_size", "16777216"}, + {ScopeGlobal | ScopeSession, TmpTableSize, "16777216"}, {ScopeGlobal, "innodb_max_purge_lag", "0"}, {ScopeGlobal | ScopeSession, "preload_buffer_size", "32768"}, {ScopeGlobal, "slave_checkpoint_period", "300"}, @@ -163,7 +163,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "innodb_flush_log_at_timeout", "1"}, {ScopeGlobal, "innodb_max_undo_log_size", ""}, {ScopeGlobal | ScopeSession, "range_alloc_block_size", "4096"}, - {ScopeGlobal, "connect_timeout", "10"}, + {ScopeGlobal, ConnectTimeout, "10"}, {ScopeGlobal | ScopeSession, "collation_server", charset.CollationUTF8}, {ScopeNone, "have_rtree_keys", "YES"}, {ScopeGlobal, "innodb_old_blocks_pct", "37"}, @@ -745,6 +745,16 @@ const ( WarningCount = "warning_count" // ErrorCount is the name for 'error_count' system variable. ErrorCount = "error_count" + // SQLSelectLimit is the name for 'sql_select_limit' system variable. + SQLSelectLimit = "sql_select_limit" + // MaxConnectErrors is the name for 'max_connect_errors' system variable. + MaxConnectErrors = "max_connect_errors" + // TableDefinitionCache is the name for 'table_definition_cache' system variable. + TableDefinitionCache = "table_definition_cache" + // TmpTableSize is the name for 'tmp_table_size' system variable. + TmpTableSize = "tmp_table_size" + // ConnectTimeout is the name for 'connect_timeout' system variable. + ConnectTimeout = "connect_timeout" ) // GlobalVarAccessor is the interface for accessing global scope system and status variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 492125fc09c50..c11f8501db71a 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -16,6 +16,7 @@ package variable import ( "encoding/json" "fmt" + "math" "strconv" "strings" "sync/atomic" @@ -27,6 +28,9 @@ import ( "github.com/pingcap/tidb/types" ) +// secondsPerYear represents seconds in a normal year. Leap year is not considered here. +const secondsPerYear = 60 * 60 * 24 * 365 + // SetDDLReorgWorkerCounter sets ddlReorgWorkerCounter count. // Max worker count is maxDDLReorgWorkerCount. func SetDDLReorgWorkerCounter(cnt int32) { @@ -161,6 +165,46 @@ func ValidateGetSystemVar(name string, isGlobal bool) error { return nil } +func checkUInt64SystemVar(name, value string, min, max uint64, vars *SessionVars) (string, error) { + if value[0] == '-' { + _, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenByArgs(name) + } + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) + return fmt.Sprintf("%d", min), nil + } + val, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenByArgs(name) + } + if val < min { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) + return fmt.Sprintf("%d", min), nil + } + if val > max { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) + return fmt.Sprintf("%d", max), nil + } + return value, nil +} + +func checkInt64SystemVar(name, value string, min, max int64, vars *SessionVars) (string, error) { + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenByArgs(name) + } + if val < min { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) + return fmt.Sprintf("%d", min), nil + } + if val > max { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) + return fmt.Sprintf("%d", max), nil + } + return value, nil +} + // ValidateSetSystemVar checks if system variable satisfies specific restriction. func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, error) { if strings.EqualFold(value, "DEFAULT") { @@ -170,19 +214,10 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, UnknownSystemVar.GenByArgs(name) } switch name { + case ConnectTimeout: + return checkUInt64SystemVar(name, value, 2, secondsPerYear, vars) case DefaultWeekFormat: - val, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "0", nil - } - if val > 7 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "7", nil - } + return checkUInt64SystemVar(name, value, 0, 7, vars) case DelayKeyWrite: if strings.EqualFold(value, "ON") || value == "1" { return "ON", nil @@ -193,103 +228,25 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, } return value, ErrWrongValueForVar.GenByArgs(name, value) case FlushTime: - val, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "0", nil - } + return checkUInt64SystemVar(name, value, 0, secondsPerYear, vars) case GroupConcatMaxLen: - val, err := strconv.ParseUint(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } // The reasonable range of 'group_concat_max_len' is 4~18446744073709551615(64-bit platforms) // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len for details - if val < 4 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "4", nil - } - if val > 18446744073709551615 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "18446744073709551615", nil - } + return checkUInt64SystemVar(name, value, 4, math.MaxUint64, vars) case InteractiveTimeout: - val, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 1 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "1", nil - } + return checkUInt64SystemVar(name, value, 1, secondsPerYear, vars) case MaxConnections: - val, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 1 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "1", nil - } - if val > 100000 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "100000", nil - } + return checkUInt64SystemVar(name, value, 1, 100000, vars) + case MaxConnectErrors: + return checkUInt64SystemVar(name, value, 1, math.MaxUint64, vars) case MaxSortLength: - val, err := strconv.ParseInt(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 4 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "4", nil - } - if val > 8388608 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "8388608", nil - } + return checkUInt64SystemVar(name, value, 4, 8388608, vars) case MaxSpRecursionDepth: - val, err := strconv.ParseInt(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "0", nil - } - if val > 255 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "255", nil - } - case OldPasswords: - val, err := strconv.Atoi(value) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "0", nil - } - if val > 2 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "2", nil - } + return checkUInt64SystemVar(name, value, 0, 255, vars) case MaxUserConnections: - val, err := strconv.ParseUint(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenByArgs(name) - } - if val < 0 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "0", nil - } - if val > 4294967295 { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenByArgs(name, value)) - return "4294967295", nil - } + return checkUInt64SystemVar(name, value, 0, 4294967295, vars) + case OldPasswords: + return checkUInt64SystemVar(name, value, 0, 2, vars) case SessionTrackGtids: if strings.EqualFold(value, "OFF") || value == "0" { return "OFF", nil @@ -299,6 +256,12 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return "ALL_GTIDS", nil } return value, ErrWrongValueForVar.GenByArgs(name, value) + case SQLSelectLimit: + return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) + case TableDefinitionCache: + return checkUInt64SystemVar(name, value, 400, 524288, vars) + case TmpTableSize: + return checkUInt64SystemVar(name, value, 1024, math.MaxUint64, vars) case TimeZone: if strings.EqualFold(value, "SYSTEM") { return "SYSTEM", nil From 5754f624c90bc2e898aeaa26737b47f02763bfb2 Mon Sep 17 00:00:00 2001 From: Du Chuan Date: Mon, 20 Aug 2018 12:08:23 +0800 Subject: [PATCH 79/87] expression: fix builtin 'CharLength' for binary string input (#7410) --- expression/builtin_string.go | 26 +++++++++++++++++++++++++- expression/builtin_string_test.go | 27 +++++++++++++++++++++++++++ expression/integration_test.go | 5 +++++ 3 files changed, 57 insertions(+), 1 deletion(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index f63ea4fe483e7..f29a950fbd9b1 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -2160,10 +2160,34 @@ func (c *charLengthFunctionClass) getFunction(ctx sessionctx.Context, args []Exp return nil, errors.Trace(argsErr) } bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + if types.IsBinaryStr(args[0].GetType()) { + sig := &builtinCharLengthBinarySig{bf} + return sig, nil + } sig := &builtinCharLengthSig{bf} return sig, nil } +type builtinCharLengthBinarySig struct { + baseBuiltinFunc +} + +func (b *builtinCharLengthBinarySig) Clone() builtinFunc { + newSig := &builtinCharLengthBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalInt evals a builtinCharLengthSig for binary string type. +// See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_char-length +func (b *builtinCharLengthBinarySig) evalInt(row chunk.Row) (int64, bool, error) { + val, isNull, err := b.args[0].EvalString(b.ctx, row) + if isNull || err != nil { + return 0, isNull, errors.Trace(err) + } + return int64(len(val)), false, nil +} + type builtinCharLengthSig struct { baseBuiltinFunc } @@ -2174,7 +2198,7 @@ func (b *builtinCharLengthSig) Clone() builtinFunc { return newSig } -// evalInt evals a builtinCharLengthSig. +// evalInt evals a builtinCharLengthSig for non-binary string type. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_char-length func (b *builtinCharLengthSig) evalInt(row chunk.Row) (int64, bool, error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 09e6d961e4fd1..c2202ca4661dd 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -1221,6 +1221,33 @@ func (s *testEvaluatorSuite) TestCharLength(c *C) { c.Assert(err, IsNil) c.Assert(r, testutil.DatumEquals, types.NewDatum(v.result)) } + + // Test binary string + tbl = []struct { + input interface{} + result interface{} + }{ + {"33", 2}, // string + {"你好", 6}, // mb string + {"CAFÉ", 5}, // mb string + {"", 0}, // mb string + {nil, nil}, // nil + } + for _, v := range tbl { + fc := funcs[ast.CharLength] + arg := s.datumsToConstants(types.MakeDatums(v.input)) + tp := arg[0].GetType() + tp.Tp = mysql.TypeVarString + tp.Charset = charset.CharsetBin + tp.Collate = charset.CollationBin + tp.Flen = types.UnspecifiedLength + tp.Flag = mysql.BinaryFlag + f, err := fc.getFunction(s.ctx, arg) + c.Assert(err, IsNil) + r, err := evalBuiltinFunc(f, chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(r, testutil.DatumEquals, types.NewDatum(v.result)) + } } func (s *testEvaluatorSuite) TestFindInSet(c *C) { diff --git a/expression/integration_test.go b/expression/integration_test.go index 52ee004d0e1f4..e57b5ef431b3f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -827,6 +827,11 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { // for char_length result = tk.MustQuery(`select char_length(null), char_length("Hello"), char_length("a中b文c"), char_length(123),char_length(12.3456);`) result.Check(testkit.Rows(" 5 5 3 7")) + result = tk.MustQuery(`select char_length(null), char_length("Hello"), char_length("a 中 b 文 c"), char_length("НОЧЬ НА ОКРАИНЕ МОСКВЫ");`) + result.Check(testkit.Rows(" 5 9 22")) + // for char_length, binary string type + result = tk.MustQuery(`select char_length(null), char_length(binary("Hello")), char_length(binary("a 中 b 文 c")), char_length(binary("НОЧЬ НА ОКРАИНЕ МОСКВЫ"));`) + result.Check(testkit.Rows(" 5 13 41")) // for elt result = tk.MustQuery(`select elt(0, "abc", "def"), elt(2, "hello", "中文", "tidb"), elt(4, "hello", "中文", From 7773b283907017ccf4835e22091c7a0508cf5383 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 20 Aug 2018 14:09:26 +0800 Subject: [PATCH 80/87] *: add a String method for Datum to make the log more friendly (#7426) --- executor/insert.go | 2 ++ executor/replace.go | 2 ++ executor/write_test.go | 35 +++++++++++++++++++++ store/mockstore/mocktikv/cop_handler_dag.go | 3 +- types/datum.go | 10 ++++++ 5 files changed, 50 insertions(+), 2 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index c69d4fafec8a9..2c8ce8c27ac3b 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -178,6 +179,7 @@ func (e *InsertExec) Open(ctx context.Context) error { func (e *InsertExec) updateDupRow(row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { oldRow, err := e.getOldRow(e.ctx, e.Table, handle) if err != nil { + log.Errorf("[insert on dup] cannot find the record whose handle is %d for the to-be-inserted row %v", handle, row.row) return errors.Trace(err) } // Do update row. diff --git a/executor/replace.go b/executor/replace.go index 595c473e8a34b..200b6343aa84d 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -52,6 +53,7 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { + log.Errorf("[replace] cannot find the record whose handle is %d for the to-be-inserted row %v", handle, r.row) return false, errors.Trace(err) } rowUnchanged, err := types.EqualDatums(e.ctx.GetSessionVars().StmtCtx, oldRow, newRow) diff --git a/executor/write_test.go b/executor/write_test.go index 9d97a4ba5d7d0..c7476e9a8b3ae 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -28,8 +28,11 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" + "golang.org/x/net/context" ) type testBypassSuite struct{} @@ -1996,3 +1999,35 @@ func (s *testSuite) TestUpdateAffectRowCnt(c *C) { ctx = tk.Se.(sessionctx.Context) c.Assert(ctx.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(2)) } + +func (s *testSuite) TestReplaceLog(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) + + // Make some dangling index. + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("testLog") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + tblInfo := tbl.Meta() + idxInfo := findIndexByName("b", tblInfo.Indices) + indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), 1) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec(`replace into testLog values (0, 0), (1, 1);`) + c.Assert(err, NotNil) + expErr := errors.New(`can not be duplicated row, due to old row not found. handle 1 not found`) + c.Assert(expErr.Error() == err.Error(), IsTrue, Commentf("obtained error: (%s)\nexpected error: (%s)", err.Error(), expErr.Error())) + + tk.MustQuery(`admin cleanup index testLog b;`).Check(testkit.Rows("1")) +} diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 861d274c7832c..456e1feb057b2 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -170,9 +170,8 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex func constructTimeZone(name string, offset int) (*time.Location, error) { if name != "" { return LocCache.getLoc(name) - } else { - return time.FixedZone("", offset), nil } + return time.FixedZone("", offset), nil } func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Request) (tikvpb.Tikv_CoprocessorStreamClient, error) { diff --git a/types/datum.go b/types/datum.go index cf4e886a0f260..c978947da6016 100644 --- a/types/datum.go +++ b/types/datum.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" + log "github.com/sirupsen/logrus" ) // Kind constants. @@ -1871,3 +1872,12 @@ func CopyRow(dr []Datum) []Datum { } return c } + +// String implements fmt.Stringer interface. +func (d Datum) String() string { + str, err := d.ToString() + if err != nil { + log.Info(err) + } + return fmt.Sprintf("(Kind: %s, Value: %s)", kind2Str[d.k], str) +} From b1aef891446bb16f8ee72515efd81303428eb967 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 20 Aug 2018 14:51:12 +0800 Subject: [PATCH 81/87] stats: fix the data race in test (#7434) --- statistics/update_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/statistics/update_test.go b/statistics/update_test.go index 5d41462039908..1770c87cf183d 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -39,10 +39,13 @@ var _ = Suite(&testStatsUpdateSuite{}) type testStatsUpdateSuite struct { store kv.Storage do *domain.Domain + hook logHook } func (s *testStatsUpdateSuite) SetUpSuite(c *C) { testleak.BeforeTest() + // Add the hook here to avoid data race. + log.AddHook(&s.hook) var err error s.store, s.do, err = newStoreWithBootstrap(0) c.Assert(err, IsNil) @@ -796,11 +799,9 @@ func (s *testStatsUpdateSuite) TestLogDetailedInfo(c *C) { }, } log.SetLevel(log.DebugLevel) - var hook logHook - log.AddHook(&hook) for _, t := range tests { - hook.results = "" + s.hook.results = "" testKit.MustQuery(t.sql) - c.Assert(hook.results, Equals, t.result) + c.Assert(s.hook.results, Equals, t.result) } } From 233660af6985ec3eac0b5d50d14e28bc1020b376 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 20 Aug 2018 15:17:17 +0800 Subject: [PATCH 82/87] Revert "*: add a String method for Datum to make the log more friendly (#7426)" (#7435) This reverts commit 7773b283907017ccf4835e22091c7a0508cf5383. --- executor/insert.go | 2 -- executor/replace.go | 2 -- executor/write_test.go | 35 --------------------- store/mockstore/mocktikv/cop_handler_dag.go | 3 +- types/datum.go | 10 ------ 5 files changed, 2 insertions(+), 50 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index 2c8ce8c27ac3b..c69d4fafec8a9 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -179,7 +178,6 @@ func (e *InsertExec) Open(ctx context.Context) error { func (e *InsertExec) updateDupRow(row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { oldRow, err := e.getOldRow(e.ctx, e.Table, handle) if err != nil { - log.Errorf("[insert on dup] cannot find the record whose handle is %d for the to-be-inserted row %v", handle, row.row) return errors.Trace(err) } // Do update row. diff --git a/executor/replace.go b/executor/replace.go index 200b6343aa84d..595c473e8a34b 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -53,7 +52,6 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { - log.Errorf("[replace] cannot find the record whose handle is %d for the to-be-inserted row %v", handle, r.row) return false, errors.Trace(err) } rowUnchanged, err := types.EqualDatums(e.ctx.GetSessionVars().StmtCtx, oldRow, newRow) diff --git a/executor/write_test.go b/executor/write_test.go index c7476e9a8b3ae..9d97a4ba5d7d0 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -28,11 +28,8 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" - "golang.org/x/net/context" ) type testBypassSuite struct{} @@ -1999,35 +1996,3 @@ func (s *testSuite) TestUpdateAffectRowCnt(c *C) { ctx = tk.Se.(sessionctx.Context) c.Assert(ctx.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(2)) } - -func (s *testSuite) TestReplaceLog(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) - - // Make some dangling index. - s.ctx = mock.NewContext() - s.ctx.Store = s.store - is := s.domain.InfoSchema() - dbName := model.NewCIStr("test") - tblName := model.NewCIStr("testLog") - tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) - tblInfo := tbl.Meta() - idxInfo := findIndexByName("b", tblInfo.Indices) - indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), 1) - c.Assert(err, IsNil) - err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - - _, err = tk.Exec(`replace into testLog values (0, 0), (1, 1);`) - c.Assert(err, NotNil) - expErr := errors.New(`can not be duplicated row, due to old row not found. handle 1 not found`) - c.Assert(expErr.Error() == err.Error(), IsTrue, Commentf("obtained error: (%s)\nexpected error: (%s)", err.Error(), expErr.Error())) - - tk.MustQuery(`admin cleanup index testLog b;`).Check(testkit.Rows("1")) -} diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 456e1feb057b2..861d274c7832c 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -170,8 +170,9 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex func constructTimeZone(name string, offset int) (*time.Location, error) { if name != "" { return LocCache.getLoc(name) + } else { + return time.FixedZone("", offset), nil } - return time.FixedZone("", offset), nil } func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Request) (tikvpb.Tikv_CoprocessorStreamClient, error) { diff --git a/types/datum.go b/types/datum.go index c978947da6016..cf4e886a0f260 100644 --- a/types/datum.go +++ b/types/datum.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" - log "github.com/sirupsen/logrus" ) // Kind constants. @@ -1872,12 +1871,3 @@ func CopyRow(dr []Datum) []Datum { } return c } - -// String implements fmt.Stringer interface. -func (d Datum) String() string { - str, err := d.ToString() - if err != nil { - log.Info(err) - } - return fmt.Sprintf("(Kind: %s, Value: %s)", kind2Str[d.k], str) -} From d567770076f118d2b5d886d54e1f811d5b6df978 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 20 Aug 2018 15:25:51 +0800 Subject: [PATCH 83/87] server: fix prepare result for zero timestamp/time/datetime (#7415) --- server/util.go | 3 --- server/util_test.go | 6 +++--- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/server/util.go b/server/util.go index f7f982512bc5a..6a38f10281b82 100644 --- a/server/util.go +++ b/server/util.go @@ -210,9 +210,6 @@ func dumpBinaryDateTime(data []byte, t types.Time, loc *time.Location) ([]byte, } year, mon, day := t.Time.Year(), t.Time.Month(), t.Time.Day() - if t.IsZero() { - year, mon, day = 1, int(time.January), 1 - } switch t.Type { case mysql.TypeTimestamp, mysql.TypeDatetime: data = append(data, 11) diff --git a/server/util_test.go b/server/util_test.go index 655b13837ead6..3f0505045f29d 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -63,18 +63,18 @@ func (s *testUtilSuite) TestDumpBinaryTime(c *C) { c.Assert(err, IsNil) d, err := dumpBinaryDateTime(nil, t, nil) c.Assert(err, IsNil) - c.Assert(d, DeepEquals, []byte{11, 1, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0}) + c.Assert(d, DeepEquals, []byte{11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}) t, err = types.ParseDatetime(nil, "0000-00-00 00:00:00.0000000") c.Assert(err, IsNil) d, err = dumpBinaryDateTime(nil, t, nil) c.Assert(err, IsNil) - c.Assert(d, DeepEquals, []byte{11, 1, 0, 1, 1, 0, 0, 0, 0, 0, 0, 0}) + c.Assert(d, DeepEquals, []byte{11, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}) t, err = types.ParseDate(nil, "0000-00-00") c.Assert(err, IsNil) d, err = dumpBinaryDateTime(nil, t, nil) c.Assert(err, IsNil) - c.Assert(d, DeepEquals, []byte{4, 1, 0, 1, 1}) + c.Assert(d, DeepEquals, []byte{4, 0, 0, 0, 0}) myDuration, err := types.ParseDuration("0000-00-00 00:00:00.0000000", 6) c.Assert(err, IsNil) From 97b60c97579d83e03a953c646869670a2d2ba20e Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 20 Aug 2018 19:06:52 +0800 Subject: [PATCH 84/87] server: fix enum type flag (#7438) --- server/column.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/column.go b/server/column.go index 27eee7fccc780..27ca3c23d734e 100644 --- a/server/column.go +++ b/server/column.go @@ -64,7 +64,7 @@ func dumpFlag(tp byte, flag uint16) uint16 { case mysql.TypeSet: return flag | uint16(mysql.SetFlag) case mysql.TypeEnum: - return flag | uint16(mysql.TypeEnum) + return flag | uint16(mysql.EnumFlag) default: return flag } From 6d0bcf50ac9eb25d2e6f8fbd686d488a701eba55 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 20 Aug 2018 23:09:00 +0800 Subject: [PATCH 85/87] makefile: add `WITH_CHECK ` os env to make and tidb-server -V print out admin check status. (#7439) --- Makefile | 9 +++++++-- util/printer/printer.go | 6 ++++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index 64c93de20d7b3..d7037427b7f49 100644 --- a/Makefile +++ b/Makefile @@ -177,11 +177,16 @@ ifeq ("$(WITH_RACE)", "1") GOBUILD = GOPATH=$(GOPATH) CGO_ENABLED=1 $(GO) build endif +CHECK_FLAG = +ifeq ("$(WITH_CHECK)", "1") + CHECK_FLAG = $(TEST_LDFLAGS) +endif + server: parserlib ifeq ($(TARGET), "") - $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -o bin/tidb-server tidb-server/main.go + $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go else - $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -o '$(TARGET)' tidb-server/main.go + $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go endif server_check: parserlib diff --git a/util/printer/printer.go b/util/printer/printer.go index b3f57cc020d8a..510c87c2592da 100644 --- a/util/printer/printer.go +++ b/util/printer/printer.go @@ -60,14 +60,16 @@ func GetTiDBInfo() string { "UTC Build Time: %s\n"+ "GoVersion: %s\n"+ "Race Enabled: %v\n"+ - "TiKV Min Version: %s", + "TiKV Min Version: %s\n"+ + "Check Table Before Drop: %v", mysql.TiDBReleaseVersion, TiDBGitHash, TiDBGitBranch, TiDBBuildTS, GoVersion, israce.RaceEnabled, - TiKVMinVersion) + TiKVMinVersion, + config.CheckTableBeforeDrop) } // checkValidity checks whether cols and every data have the same length. From d4e08853e4d4d867676f0f17ec6e5b15d5a0bdf8 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 21 Aug 2018 11:43:48 +0800 Subject: [PATCH 86/87] executor,statistics: make query feedback work for partitioned table (#7394) --- ddl/db_test.go | 2 +- executor/builder.go | 6 +-- statistics/update.go | 17 ++++++--- statistics/update_test.go | 79 +++++++++++++++++++++++++++++++++++++++ 4 files changed, 95 insertions(+), 9 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 82750f446beda..206f1b95ff85d 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3167,7 +3167,7 @@ func (s *testDBSuite) TestPartitionDropIndex(c *C) { LOOP: for { select { - case err := <-done: + case err = <-done: if err == nil { break LOOP } diff --git a/executor/builder.go b/executor/builder.go index c5761c6b98932..4d5f3e414c560 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1552,7 +1552,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plan.PhysicalTableReader) (* if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) } else { - e.feedback = statistics.NewQueryFeedback(ts.Table.ID, ts.Hist, int64(ts.StatsCount()), ts.Desc) + e.feedback = statistics.NewQueryFeedback(e.physicalTableID, ts.Hist, int64(ts.StatsCount()), ts.Desc) } collect := e.feedback.CollectFeedback(len(ts.Ranges)) e.dagPB.CollectRangeCounts = &collect @@ -1609,7 +1609,7 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plan.PhysicalIndexReader) (* if containsLimit(dagReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { - e.feedback = statistics.NewQueryFeedback(is.Table.ID, is.Hist, int64(is.StatsCount()), is.Desc) + e.feedback = statistics.NewQueryFeedback(e.physicalTableID, is.Hist, int64(is.StatsCount()), is.Desc) } collect := e.feedback.CollectFeedback(len(is.Ranges)) e.dagPB.CollectRangeCounts = &collect @@ -1682,7 +1682,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plan.PhysicalIndexLook if containsLimit(indexReq.Executors) { e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) } else { - e.feedback = statistics.NewQueryFeedback(is.Table.ID, is.Hist, int64(is.StatsCount()), is.Desc) + e.feedback = statistics.NewQueryFeedback(e.physicalTableID, is.Hist, int64(is.StatsCount()), is.Desc) } // do not collect the feedback for table request. collectTable := false diff --git a/statistics/update.go b/statistics/update.go index fa4f73234369c..d33100af918aa 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -505,18 +505,25 @@ func (h *Handle) HandleUpdateStats(is infoschema.InfoSchema) error { // handleSingleHistogramUpdate updates the Histogram and CM Sketch using these feedbacks. All the feedbacks for // the same index or column are gathered in `rows`. func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []chunk.Row) (err error) { - tableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2) + physicalTableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2) defer func() { if err == nil { - err = errors.Trace(h.deleteOutdatedFeedback(tableID, histID, isIndex)) + err = errors.Trace(h.deleteOutdatedFeedback(physicalTableID, histID, isIndex)) } }() - table, ok := is.TableByID(tableID) + h.mu.Lock() + table, ok := h.getTableByPhysicalID(is, physicalTableID) + h.mu.Unlock() // The table has been deleted. if !ok { return nil } - tbl := h.GetTableStats(table.Meta()) + var tbl *Table + if table.Meta().GetPartitionInfo() != nil { + tbl = h.GetPartitionStats(table.Meta(), physicalTableID) + } else { + tbl = h.GetTableStats(table.Meta()) + } var cms *CMSketch var hist *Histogram if isIndex == 1 { @@ -548,7 +555,7 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch if table.Meta().PKIsHandle && isIndex == 0 { hist.NDV = int64(hist.totalRowCount()) } - err = h.dumpStatsUpdateToKV(tableID, isIndex, q, hist, cms) + err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms) return errors.Trace(err) } diff --git a/statistics/update_test.go b/statistics/update_test.go index 1770c87cf183d..4e42383946fc1 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -636,6 +636,85 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) } +func (s *testStatsUpdateSuite) TestQueryFeedbackForPartition(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("set @@session.tidb_enable_table_partition=1") + testKit.MustExec(`create table t (a bigint(64), b bigint(64), primary key(a), index idx(b)) + partition by range (a) ( + partition p0 values less than (3), + partition p1 values less than (6))`) + testKit.MustExec("insert into t values (1,2),(2,2),(3,4),(4,1),(5,6)") + testKit.MustExec("analyze table t") + + oriProbability := statistics.FeedbackProbability + defer func() { + statistics.FeedbackProbability = oriProbability + }() + h := s.do.StatsHandle() + statistics.FeedbackProbability = 1 + tests := []struct { + sql string + hist string + idxCols int + }{ + { + // test primary key feedback + sql: "select * from t where t.a <= 5", + hist: "column:1 ndv:2 totColSize:0\n" + + "num: 1 lower_bound: -9223372036854775808 upper_bound: 1 repeats: 0\n" + + "num: 1 lower_bound: 2 upper_bound: 5 repeats: 0", + idxCols: 0, + }, + { + // test index feedback by double read + sql: "select * from t use index(idx) where t.b <= 5", + hist: "index:1 ndv:1\n" + + "num: 2 lower_bound: -inf upper_bound: 6 repeats: 0", + idxCols: 1, + }, + { + // test index feedback by single read + sql: "select b from t use index(idx) where t.b <= 5", + hist: "index:1 ndv:1\n" + + "num: 2 lower_bound: -inf upper_bound: 6 repeats: 0", + idxCols: 1, + }, + } + is := s.do.InfoSchema() + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tblInfo := table.Meta() + pi := tblInfo.GetPartitionInfo() + c.Assert(pi, NotNil) + + // This test will check the result of partition p0. + var pid int64 + for _, def := range pi.Definitions { + if def.Name.L == "p0" { + pid = def.ID + break + } + } + + for i, t := range tests { + testKit.MustQuery(t.sql) + c.Assert(h.DumpStatsDeltaToKV(statistics.DumpAll), IsNil) + c.Assert(h.DumpStatsFeedbackToKV(), IsNil) + c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) + c.Assert(err, IsNil) + h.Update(is) + tbl := h.GetPartitionStats(tblInfo, pid) + if t.idxCols == 0 { + c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[i].hist) + } else { + c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[i].hist) + } + } + testKit.MustExec("drop table t") +} + func (s *testStatsUpdateSuite) TestUpdateSystemTable(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) From 1dc41d8b79cd68d6f218070b446ada2fd2a8c6ae Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 21 Aug 2018 12:43:41 +0800 Subject: [PATCH 87/87] add logs for replace and insert on dup (#7436) --- executor/insert.go | 2 ++ executor/replace.go | 2 ++ executor/write_test.go | 35 +++++++++++++++++++++ store/mockstore/mocktikv/cop_handler_dag.go | 3 +- types/datum.go | 9 ++++++ 5 files changed, 49 insertions(+), 2 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index c69d4fafec8a9..66a91ee65830c 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -178,6 +179,7 @@ func (e *InsertExec) Open(ctx context.Context) error { func (e *InsertExec) updateDupRow(row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { oldRow, err := e.getOldRow(e.ctx, e.Table, handle) if err != nil { + log.Errorf("[insert on dup] handle is %d for the to-be-inserted row %s", handle, types.DatumsToStrNoErr(row.row)) return errors.Trace(err) } // Do update row. diff --git a/executor/replace.go b/executor/replace.go index 595c473e8a34b..a37f22dcc7e79 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -52,6 +53,7 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := e.batchChecker.getOldRow(e.ctx, r.t, handle) if err != nil { + log.Errorf("[replace] handle is %d for the to-be-inserted row %v", handle, types.DatumsToStrNoErr(r.row)) return false, errors.Trace(err) } rowUnchanged, err := types.EqualDatums(e.ctx.GetSessionVars().StmtCtx, oldRow, newRow) diff --git a/executor/write_test.go b/executor/write_test.go index 9d97a4ba5d7d0..c7476e9a8b3ae 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -28,8 +28,11 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" + "golang.org/x/net/context" ) type testBypassSuite struct{} @@ -1996,3 +1999,35 @@ func (s *testSuite) TestUpdateAffectRowCnt(c *C) { ctx = tk.Se.(sessionctx.Context) c.Assert(ctx.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(2)) } + +func (s *testSuite) TestReplaceLog(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) + + // Make some dangling index. + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("testLog") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + tblInfo := tbl.Meta() + idxInfo := findIndexByName("b", tblInfo.Indices) + indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), 1) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec(`replace into testLog values (0, 0), (1, 1);`) + c.Assert(err, NotNil) + expErr := errors.New(`can not be duplicated row, due to old row not found. handle 1 not found`) + c.Assert(expErr.Error() == err.Error(), IsTrue, Commentf("obtained error: (%s)\nexpected error: (%s)", err.Error(), expErr.Error())) + + tk.MustQuery(`admin cleanup index testLog b;`).Check(testkit.Rows("1")) +} diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index 861d274c7832c..456e1feb057b2 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -170,9 +170,8 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex func constructTimeZone(name string, offset int) (*time.Location, error) { if name != "" { return LocCache.getLoc(name) - } else { - return time.FixedZone("", offset), nil } + return time.FixedZone("", offset), nil } func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Request) (tikvpb.Tikv_CoprocessorStreamClient, error) { diff --git a/types/datum.go b/types/datum.go index cf4e886a0f260..4788318f4ff21 100644 --- a/types/datum.go +++ b/types/datum.go @@ -25,6 +25,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" @@ -1857,6 +1858,14 @@ func DatumsToString(datums []Datum, handleSpecialValue bool) (string, error) { return strings.Join(strs, ", "), nil } +// DatumsToStrNoErr converts some datums to a formatted string. +// If an error occurs, it will print a log instead of returning an error. +func DatumsToStrNoErr(datums []Datum) string { + str, err := DatumsToString(datums, true) + terror.Log(errors.Trace(err)) + return str +} + // CopyDatum returns a new copy of the datum. // TODO: Abandon this function. func CopyDatum(datum Datum) Datum {