From dac8b98773c108b3cd1f7266145eb50c06bcc651 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 28 Jun 2019 14:53:19 +0800 Subject: [PATCH 01/21] gc_worker: add test for check save gc safe point (#10964) (#10967) --- store/mockstore/mocktikv/pd.go | 2 +- store/tikv/gcworker/gc_worker_test.go | 31 ++++++++++++++++++++++++++- 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/store/mockstore/mocktikv/pd.go b/store/mockstore/mocktikv/pd.go index e79f1a471164a..6e9097f4aebbf 100644 --- a/store/mockstore/mocktikv/pd.go +++ b/store/mockstore/mocktikv/pd.go @@ -103,7 +103,7 @@ func (c *pdClient) GetAllStores(ctx context.Context, opts ...pd.GetStoreOption) } func (c *pdClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { - panic("unimplemented") + return 0, nil } func (c *pdClient) Close() { diff --git a/store/tikv/gcworker/gc_worker_test.go b/store/tikv/gcworker/gc_worker_test.go index 357ae13bcdfe0..e0c3664bc45a3 100644 --- a/store/tikv/gcworker/gc_worker_test.go +++ b/store/tikv/gcworker/gc_worker_test.go @@ -23,6 +23,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" + pd "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" @@ -42,6 +43,7 @@ type testGCWorkerSuite struct { oracle *mockoracle.MockOracle gcWorker *GCWorker dom *domain.Domain + pdClient pd.Client } var _ = Suite(&testGCWorkerSuite{}) @@ -60,7 +62,8 @@ func (s *testGCWorkerSuite) SetUpTest(c *C) { s.dom, err = session.BootstrapSession(s.store) c.Assert(err, IsNil) - gcWorker, err := NewGCWorker(s.store, mocktikv.NewPDClient(s.cluster)) + s.pdClient = mocktikv.NewPDClient(s.cluster) + gcWorker, err := NewGCWorker(s.store, s.pdClient) c.Assert(err, IsNil) gcWorker.Start() gcWorker.Close() @@ -324,3 +327,29 @@ func (s *testGCWorkerSuite) TestCheckGCMode(c *C) { c.Assert(err, IsNil) c.Assert(useDistributedGC, Equals, true) } + +func (s *testGCWorkerSuite) TestRunGCJob(c *C) { + gcSafePointCacheInterval = 0 + err := RunGCJob(context.Background(), s.store, 0, "mock", 1) + c.Assert(err, IsNil) + gcWorker, err := NewGCWorker(s.store, s.pdClient) + c.Assert(err, IsNil) + gcWorker.Start() + useDistributedGC, err := gcWorker.(*GCWorker).checkUseDistributedGC() + c.Assert(useDistributedGC, IsTrue) + c.Assert(err, IsNil) + safePoint := uint64(time.Now().Unix()) + gcWorker.(*GCWorker).runGCJob(context.Background(), safePoint, 1) + getSafePoint, err := loadSafePoint(gcWorker.(*GCWorker).store.GetSafePointKV()) + c.Assert(err, IsNil) + c.Assert(getSafePoint, Equals, safePoint) + gcWorker.Close() +} + +func loadSafePoint(kv tikv.SafePointKV) (uint64, error) { + val, err := kv.Get(tikv.GcSavedSafePoint) + if err != nil { + return 0, err + } + return strconv.ParseUint(val, 10, 64) +} From 4a6ae82573989ac293611a3597bafee8ac996b73 Mon Sep 17 00:00:00 2001 From: winkyao Date: Sun, 30 Jun 2019 09:33:22 +0800 Subject: [PATCH 02/21] CHANGELOG: add release notes for v3.0.0,v3.0.0-rc.3 (#10983) --- CHANGELOG.md | 142 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 142 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a510f9dd5e47e..0c38c82bce51f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,148 @@ # TiDB Changelog All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/pingcap/pd/blob/master/CHANGELOG.md). +## [3.0.0] 2019-06-28 +## New Features +* Support Window Functions; compatible with all window functions in MySQL 8.0, including `NTILE`, `LEAD`, `LAG`, `PERCENT_RANK`, `NTH_VALUE`, `CUME_DIST`, `FIRST_VALUE` , `LAST_VALUE`, `RANK`, `DENSE_RANK`, and `ROW_NUMBER` +* Support Views (Experimental) +* Improve Table Partition + - Support Range Partition + - Support Hash Partition +* Add the plug-in framework, supporting plugins such as IP Whitelist (Enterprise feature) and Audit Log (Enterprise feature). +* Support the SQL Plan Management function to create SQL execution plan binding to ensure query stability (Experimental) + +## SQL Optimizer +* Optimize the `NOT EXISTS` subquery and convert it to `Anti Semi Join` to improve performance +* Optimize the constant propagation on the `Outer Join`, and add the optimization rule of `Outer Join` elimination to reduce non-effective computations and improve performance +* Optimize the `IN` subquery to execute `Inner Join` after aggregation to improve performance +* Optimize `Index Join` to adapt to more scenarios +* Improve the Partition Pruning optimization rule of Range Partition +* Optimize the query logic for `_tidb_rowid`to avoid full table scan and improve performance +* Match more prefix columns of the indexes when extracting access conditions of composite indexes if there are relevant columns in the filter to improve performance +* Improve the accuracy of cost estimates by using order correlation between columns +* Optimize `Join Reorder` based on the Greedy algorithm and the dynamic planning algorithm to improve accuracy for index selection using `Join` +* Support Skyline Pruning, with some rules to prevent the execution plan from relying too heavily on statistics to improve query stability +* Improve the accuracy of row count estimation for single-column indexes with NULL values +* Support `FAST ANALYZE` that randomly samples in each Region to avoid full table scan and improve performance with statistics collection +* Support the incremental Analyze operation on monotonically increasing index columns to improve performance with statistics collection +* Support using subqueries in the `DO` statement +* Support using `Index Join` in transactions +* Optimize `prepare`/`execute` to support DDL statements with no parameters +* Modify the system behaviour to auto load statistics when the `stats-lease` variable value is 0 +* Support exporting historical statistics +* Support the `dump`/`load` correlation of histograms + +## SQL Execution Engine +* Optimize log output: `EXECUTE` outputs user variables and `COMMIT` outputs slow query logs to facilitate troubleshooting +* Support the `EXPLAIN ANALYZE` function to improve SQL tuning usability +* Support the `admin show next_row_id` command to get the ID of the next row +* Add six built-in functions: `JSON_QUOTE`, `JSON_ARRAY_APPEND`, `JSON_MERGE_PRESERVE`, `BENCHMARK` ,`COALESCE`, and `NAME_CONST` +* Optimize control logics on the chunk size to dynamically adjust based on the query context, to reduce the SQL execution time and resource consumption +* Support tracking and controlling memory usage in three operators - `TableReader`, `IndexReader` and `IndexLookupReader` +* Optimize the Merge Join operator to support an empty `ON` condition +* Optimize write performance for single tables that contains too many columns +* Improve the performance of `admin show ddl jobs` by supporting scanning data in reverse order +* Add the `split table region` statement to manually split the table Region to alleviate the hotspot issue +* Add the `split index region` statement to manually split the index Region to alleviate the hotspot issue +* Add a blacklist to prohibit pushing down expressions to Coprocessor +* Optimize the `Expensive Query` log to print the SQL query in the log when it exceeds the configured limit of execution time or memory + +## DDL +* Support migrating from character set `utf8` to `utf8mb4` +* Change the default character set from`utf8` to `utf8mb4` +* Add the `alter schema` statement to modify the character set and the collation of the database +* Support ALTER algorithm `INPLACE`/`INSTANT` +* Support `SHOW CREATE VIEW` +* Support `SHOW CREATE USER` +* Support fast recovery of mistakenly deleted tables +* Support adjusting the number of concurrencies of ADD INDEX dynamically +* Add the `pre_split_regions` option that pre-allocates Regions when creating the table using the `CREATE TABLE` statement, to relieve write hot Regions caused by lots of writes after the table creation +* Support splitting Regions by the index and range of the table specified using SQL statements to relieve hotspot issues +* Add the `ddl_error_count_limit` global variable to limit the number of DDL task retries +* Add a feature to use `SHARD_ROW_ID_BITS` to scatter row IDs when the column contains an AUTO_INCREMENT attribute to relieve the hotspot issue +* Optimize the lifetime of invalid DDL metadata to speed up recovering the normal execution of DDL operations after upgrading the TiDB cluster + +## Transactions +* Support the pessimistic transaction model (Experimental) +* Optimize transaction processing logics to adapt to more scenarios: + - Change the default value `tidb_disable_txn_auto_retry` to `on`, which means non-auto committed transactions will not be retried + - Add the `tidb_batch_commit` system variable to split a transaction into multiple ones to be executed concurrently + - Add the `tidb_low_resolution_tso` system variable to control the number of TSOs to obtain in batches and reduce the number of times that transactions request for TSOs, to improve performance in scenarios with relatively low requirement of consistency + - Add the `tidb_skip_isolation_level_check` variable to control whether to report errors when the isolation level is set to SERIALIZABLE + - Modify the `tidb_disable_txn_auto_retry` system variable to make it work on all retryable errors + +## Permission Management +* Perform permission check on the `ANALYZE`, `USE`, `SET GLOBAL`, and `SHOW PROCESSLIST` statements +* Support Role Based Access Control (RBAC) (Experimental) + +## Server +* Optimize slow query logs + - Restructure the log format + - Optimize the log content + - Optimize the log query method to support using the `INFORMATION_SCHEMA.SLOW_QUERY` and `ADMIN SHOW SLOW` statements of the memory table to query slow query logs +* Develop a unified log format specification with restructured log system to facilitate collection and analysis by tools +* Support using SQL statements to manage Binlog services, including querying status, enabling Binlog, maintaining and sending Binlog strategies. +* Support using `unix_socket` to connect to the database +* Support `Trace` for SQL statements +* Support getting information for a TiDB instance via the `/debug/zip` HTTP interface to facilitate troubleshooting. +* Optimize monitoring items to facilitate troubleshooting: + - Add the `high_error_rate_feedback_total` monitoring item to monitor the difference between the actual data volume and the estimated data volume based on statistics + - Add a QPS monitoring item in the database dimension +* Optimize the system initialization process to only allow the DDL owner to perform the initialization. This reduces the startup time for initialization or upgrading. +* Optimize the execution logic of `kill query` to improve performance and ensure resource is release properly +* Add a startup option `config-check` to check the validity of the configuration file +* Add the `tidb_back_off_weight` system variable to control the backoff time of internal error retries +* Add the `wait_timeout`and `interactive_timeout` system variables to control the maximum idle connections allowed +* Add the connection pool for TiKV to shorten the connection establishing time + +## Compatibility +* Support the `ALLOW_INVALID_DATES` SQL mode +* Support the MySQL 320 Handshake protocol +* Support manifesting unsigned BIGINT columns as auto-increment columns +* Support the `SHOW CREATE DATABASE IF NOT EXISTS` syntax +* Optimize the fault tolerance of `load data` for CSV files +* Abandon the predicate pushdown operation when the filtering condition contains a user variable to improve the compatibility with MySQL’s behavior of using user variables to simulate Window Functions + + +## [3.0.0-rc.3] 2019-06-21 +## SQL Optimizer +* Remove the feature of collecting virtual generated column statistics[#10629](https://github.com/pingcap/tidb/pull/10629) +* Fix the issue that the primary key constant overflows during point queries [#10699](https://github.com/pingcap/tidb/pull/10699) +* Fix the issue that using uninitialized information in `fast analyze` causes panic [#10691](https://github.com/pingcap/tidb/pull/10691) +* Fix the issue that executing the `create view` statement using `prepare` causes panic because of wrong column information [#10713](https://github.com/pingcap/tidb/pull/10713) +* Fix the issue that the column information is not cloned when handling window functions [#10720](https://github.com/pingcap/tidb/pull/10720) +* Fix the wrong estimation for the selectivity rate of the inner table selection in index join [#10854](https://github.com/pingcap/tidb/pull/10854) +* Support automatic loading statistics when the `stats-lease` variable value is 0 [#10811](https://github.com/pingcap/tidb/pull/10811) + +## Execution Engine +* Fix the issue that resources are not correctly released when calling the `Close` function in `StreamAggExec` [#10636](https://github.com/pingcap/tidb/pull/10636) +* Fix the issue that the order of `table_option` and `partition_options` is incorrect in the result of executing the `show create table` statement for partitioned tables [#10689](https://github.com/pingcap/tidb/pull/10689) +* Improve the performance of `admin show ddl jobs` by supporting scanning data in reverse order [#10687](https://github.com/pingcap/tidb/pull/10687) +* Fix the issue that the result of the `show grants` statement in RBAC is incompatible with that of MySQL when this statement has the `current_user` field [#10684](https://github.com/pingcap/tidb/pull/10684) +* Fix the issue that UUIDs might generate duplicate values ​​on multiple nodes [#10712](https://github.com/pingcap/tidb/pull/10712) +* Fix the issue that the `show view` privilege is not considered in `explain` [#10635](https://github.com/pingcap/tidb/pull/10635) +* Add the `split table region` statement to manually split the table Region to alleviate the hotspot issue [#10765](https://github.com/pingcap/tidb/pull/10765) +* Add the `split index region` statement to manually split the index Region to alleviate the hotspot issue [#10764](https://github.com/pingcap/tidb/pull/10764) +* Fix the incorrect execution issue when you execute multiple statements such as `create user`, `grant`, or `revoke` consecutively [#10737] (https://github.com/pingcap/tidb/pull/10737) +* Add a blacklist to prohibit pushing down expressions to Coprocessor [#10791](https://github.com/pingcap/tidb/pull/10791) +* Add the feature of printing the `expensive query` log when a query exceeds the memory configuration limit [#10849](https://github.com/pingcap/tidb/pull/10849) +* Add the `bind-info-lease` configuration item to control the update time of the modified binding execution plan [#10727](https://github.com/pingcap/tidb/pull/10727) +* Fix the OOM issue in high concurrent scenarios caused by the failure to quickly release Coprocessor resources, resulted from the `execdetails.ExecDetails` pointer [#10832] (https://github.com/pingcap/tidb/pull/10832) +* Fix the panic issue caused by the `kill` statement in some cases [#10876](https://github.com/pingcap/tidb/pull/10876) +## Server +* Fix the issue that goroutine might leak when repairing GC [#10683](https://github.com/pingcap/tidb/pull/10683) +* Support displaying the `host` information in slow queries [#10693](https://github.com/pingcap/tidb/pull/10693) +* Support reusing idle links that interact with TiKV [#10632](https://github.com/pingcap/tidb/pull/10632) +* Fix the support for enabling the `skip-grant-table` option in RBAC [#10738](https://github.com/pingcap/tidb/pull/10738) +* Fix the issue that `pessimistic-txn` configuration goes invalid [#10825](https://github.com/pingcap/tidb/pull/10825) +* Fix the issue that the actively cancelled ticlient requests are still retried [#10850](https://github.com/pingcap/tidb/pull/10850) +* Improve performance in the case where pessimistic transactions conflict with optimistic transactions [#10881](https://github.com/pingcap/tidb/pull/10881) +## DDL +* Fix the issue that modifying charset using `alter table` causes the `blob` type change [#10698](https://github.com/pingcap/tidb/pull/10698) +* Add a feature to use `SHARD_ROW_ID_BITS` to scatter row IDs when the column contains an `AUTO_INCREMENT` attribute to alleviate the hotspot issue [#10794](https://github.com/pingcap/tidb/pull/10794) +* Prohibit adding stored generated columns by using the `alter table` statement [#10808](https://github.com/pingcap/tidb/pull/10808) +* Optimize the invalid survival time of DDL metadata to shorten the period during which the DDL operation is slower after cluster upgrade [#10795](https://github.com/pingcap/tidb/pull/10795) + ## [3.0.0-rc.2] 2019-05-28 ### SQL Optimizer * Support Index Join in more scenarios From d8ffce9f1be95d8cc490dcc2a79b256fc165032e Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 1 Jul 2019 12:51:54 +0800 Subject: [PATCH 03/21] executor: locks key in point get executor for pessimistic transaction (#10972) (#10976) --- cmd/explaintest/r/explain_easy.result | 3 ++ cmd/explaintest/t/explain_easy.test | 1 + executor/adapter.go | 16 +++++----- executor/point_get.go | 20 +++++++++++- planner/core/point_get_plan.go | 14 ++++++++- session/pessimistic_test.go | 44 +++++++++++++++++++++++++++ 6 files changed, 89 insertions(+), 9 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index ba83ace8edb2b..9ca1062c69484 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -434,6 +434,9 @@ id count task operator info Projection_3 10000.00 root or(NULL, gt(test.t.a, 1)) └─TableReader_5 10000.00 root data:TableScan_4 └─TableScan_4 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +explain select * from t where a = 1 for update; +id count task operator info +Point_Get_1 1.00 root table:t, handle:1 drop table if exists ta, tb; create table ta (a varchar(20)); create table tb (a varchar(20)); diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index a8cd985164413..c5044c96069cd 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -79,6 +79,7 @@ drop table if exists t; create table t(a bigint primary key); explain select * from t where a = 1 and a = 2; explain select null or a > 1 from t; +explain select * from t where a = 1 for update; drop table if exists ta, tb; create table ta (a varchar(20)); diff --git a/executor/adapter.go b/executor/adapter.go index fa535d62239ff..6f6af7de5545a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -401,7 +401,12 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { for { _, err = a.handleNoDelayExecutor(ctx, e) if err != nil { - return err + // It is possible the DML has point get plan that locks the key. + e, err = a.handlePessimisticLockError(ctx, err) + if err != nil { + return err + } + continue } keys, err1 := txn.(pessimisticTxn).KeysNeedToLock() if err1 != nil { @@ -412,21 +417,18 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } forUpdateTS := txnCtx.GetForUpdateTS() err = txn.LockKeys(ctx, forUpdateTS, keys...) + if err == nil { + return nil + } e, err = a.handlePessimisticLockError(ctx, err) if err != nil { return err } - if e == nil { - return nil - } } } // handlePessimisticLockError updates TS and rebuild executor if the err is write conflict. func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, err error) (Executor, error) { - if err == nil { - return nil, nil - } txnCtx := a.Ctx.GetSessionVars().TxnCtx var newForUpdateTS uint64 if deadlock, ok := errors.Cause(err).(*tikv.ErrDeadlock); ok { diff --git a/executor/point_get.go b/executor/point_get.go index 587791fc8741e..80730ca2c48c4 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -43,7 +43,9 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { idxVals: p.IndexValues, handle: p.Handle, startTS: startTS, + lock: p.Lock, } + b.isSelectForUpdate = p.IsForUpdate e.base().initCap = 1 e.base().maxChunkSize = 1 return e @@ -60,6 +62,7 @@ type PointGetExecutor struct { startTS uint64 snapshot kv.Snapshot done bool + lock bool } // Open implements the Executor interface. @@ -95,7 +98,7 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { return err1 } if len(handleVal) == 0 { - return nil + return e.lockKeyIfNeeded(ctx, idxKey) } e.handle, err1 = tables.DecodeHandle(handleVal) if err1 != nil { @@ -122,6 +125,10 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil && !kv.ErrNotExist.Equal(err) { return err } + err = e.lockKeyIfNeeded(ctx, key) + if err != nil { + return err + } if len(val) == 0 { if e.idxInfo != nil { return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", @@ -132,6 +139,17 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { return e.decodeRowValToChunk(val, req) } +func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) error { + if e.lock { + txn, err := e.ctx.Txn(true) + if err != nil { + return err + } + return txn.LockKeys(ctx, e.ctx.GetSessionVars().TxnCtx.GetForUpdateTS(), kv.Key(key)) + } + return nil +} + func (e *PointGetExecutor) encodeIndexKey() (_ []byte, err error) { sc := e.ctx.GetSessionVars().StmtCtx for i := range e.idxVals { diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 6ad194e01a58a..6bf2b6e197066 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -48,6 +48,8 @@ type PointGetPlan struct { expr expression.Expression ctx sessionctx.Context IsTableDual bool + Lock bool + IsForUpdate bool } type nameValuePair struct { @@ -141,6 +143,10 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) Plan { tableDual.SetSchema(fp.Schema()) return tableDual.Init(ctx, &property.StatsInfo{}) } + if x.LockTp == ast.SelectLockForUpdate { + fp.Lock = true + fp.IsForUpdate = true + } return fp } case *ast.UpdateStmt: @@ -159,7 +165,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) Plan { // 3. All the columns must be public and generated. // 4. The condition is an access path that the range is a unique key. func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetPlan { - if selStmt.Having != nil || selStmt.LockTp != ast.SelectLockNone { + if selStmt.Having != nil { return nil } else if selStmt.Limit != nil { count, offset, err := extractLimitCountOffset(ctx, selStmt.Limit) @@ -452,6 +458,9 @@ func tryUpdatePointPlan(ctx sessionctx.Context, updateStmt *ast.UpdateStmt) Plan if fastSelect.IsTableDual { return PhysicalTableDual{}.Init(ctx, &property.StatsInfo{}) } + if ctx.GetSessionVars().TxnCtx.IsPessimistic { + fastSelect.Lock = true + } orderedList := buildOrderedList(ctx, fastSelect, updateStmt.List) if orderedList == nil { return nil @@ -512,6 +521,9 @@ func tryDeletePointPlan(ctx sessionctx.Context, delStmt *ast.DeleteStmt) Plan { if fastSelect.IsTableDual { return PhysicalTableDual{}.Init(ctx, &property.StatsInfo{}) } + if ctx.GetSessionVars().TxnCtx.IsPessimistic { + fastSelect.Lock = true + } delPlan := Delete{ SelectPlan: fastSelect, }.Init(ctx) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 30cd2146075e8..1fb6332b1c8b6 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -278,3 +278,47 @@ func (s *testPessimisticSuite) TestInsertOnDup(c *C) { tk.MustExec("commit") tk.MustQuery("select * from dup").Check(testkit.Rows("1 2")) } + +func (s *testPessimisticSuite) TestPointGetKeyLock(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists point") + tk.MustExec("create table point (id int primary key, u int unique, c int)") + syncCh := make(chan struct{}) + + tk.MustExec("begin pessimistic") + tk.MustExec("update point set c = c + 1 where id = 1") + tk.MustExec("delete from point where u = 2") + go func() { + tk2.MustExec("begin pessimistic") + _, err1 := tk2.Exec("insert point values (1, 1, 1)") + c.Check(kv.ErrKeyExists.Equal(err1), IsTrue) + _, err1 = tk2.Exec("insert point values (2, 2, 2)") + c.Check(kv.ErrKeyExists.Equal(err1), IsTrue) + tk2.MustExec("rollback") + <-syncCh + }() + time.Sleep(time.Millisecond * 10) + tk.MustExec("insert point values (1, 1, 1)") + tk.MustExec("insert point values (2, 2, 2)") + tk.MustExec("commit") + syncCh <- struct{}{} + + tk.MustExec("begin pessimistic") + tk.MustExec("select * from point where id = 3 for update") + tk.MustExec("select * from point where u = 4 for update") + go func() { + tk2.MustExec("begin pessimistic") + _, err1 := tk2.Exec("insert point values (3, 3, 3)") + c.Check(kv.ErrKeyExists.Equal(err1), IsTrue) + _, err1 = tk2.Exec("insert point values (4, 4, 4)") + c.Check(kv.ErrKeyExists.Equal(err1), IsTrue) + tk2.MustExec("rollback") + <-syncCh + }() + time.Sleep(time.Millisecond * 10) + tk.MustExec("insert point values (3, 3, 3)") + tk.MustExec("insert point values (4, 4, 4)") + tk.MustExec("commit") + syncCh <- struct{}{} +} From 58e6f74e0084eee8b779a1162ee7ba2816671e3f Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Mon, 1 Jul 2019 19:51:17 +0800 Subject: [PATCH 04/21] meta: make auto increment id can be adjust. (#10978) (#11006) --- ddl/db_test.go | 14 ++++++- ddl/serial_test.go | 4 ++ executor/ddl_test.go | 5 +++ executor/seqtest/seq_executor_test.go | 4 ++ meta/autoid/autoid.go | 53 ++++++++++++++++++++++----- meta/autoid/autoid_test.go | 21 +++++++++++ 6 files changed, 90 insertions(+), 11 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 74d6c6a37c7a9..df9b12597f02c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -27,6 +27,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" tmysql "github.com/pingcap/parser/mysql" @@ -2163,7 +2164,11 @@ func (s *testDBSuite4) TestComment(c *C) { s.tk.MustExec("drop table if exists ct, ct1") } -func (s *testDBSuite5) TestRebaseAutoID(c *C) { +func (s *testDBSuite4) TestRebaseAutoID(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -2758,7 +2763,12 @@ func (s *testDBSuite1) TestModifyColumnCharset(c *C) { } -func (s *testDBSuite2) TestAlterShardRowIDBits(c *C) { +func (s *testDBSuite4) TestAlterShardRowIDBits(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() + s.tk = testkit.NewTestKit(c, s.store) tk := s.tk diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 665337cdd31a5..d4f0803cc66d5 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -241,6 +241,10 @@ func (s *testSerialSuite) TestRecoverTableByJobID(c *C) { } func (s *testSerialSuite) TestRecoverTableByTableName(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_recover") tk.MustExec("use test_recover") diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 90c2ebccfe273..c29dce7a8acc8 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -22,6 +22,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -343,6 +344,10 @@ func (s *testSuite3) TestDefaultDBAfterDropCurDB(c *C) { } func (s *testSuite3) TestRenameTable(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database rename1") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index be074db360822..05a4b68638c06 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -708,6 +708,10 @@ func checkGoroutineExists(keyword string) bool { } func (s *seqTestSuite) TestAdminShowNextID(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() step := int64(10) autoIDStep := autoid.GetStep() autoid.SetStep(step) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 8f52cb1e4adbb..cccaa35875974 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -22,6 +22,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -30,6 +31,12 @@ import ( "go.uber.org/zap" ) +const ( + minStep = 1000 + maxStep = 2000000 + defaultConsumeTime = 10 * time.Second +) + // Test needs to change it, so it's a variable. var step = int64(30000) @@ -59,8 +66,10 @@ type allocator struct { end int64 store kv.Storage // dbID is current database's ID. - dbID int64 - isUnsigned bool + dbID int64 + isUnsigned bool + lastAllocTime time.Time + step int64 } // GetStep is only used by tests @@ -124,7 +133,7 @@ func (alloc *allocator) rebase4Unsigned(tableID int64, requiredBase uint64, allo uCurrentEnd := uint64(currentEnd) if allocIDs { newBase = mathutil.MaxUint64(uCurrentEnd, requiredBase) - newEnd = mathutil.MinUint64(math.MaxUint64-uint64(step), newBase) + uint64(step) + newEnd = mathutil.MinUint64(math.MaxUint64-uint64(alloc.step), newBase) + uint64(alloc.step) } else { if uCurrentEnd >= requiredBase { newBase = uCurrentEnd @@ -169,7 +178,7 @@ func (alloc *allocator) rebase4Signed(tableID, requiredBase int64, allocIDs bool } if allocIDs { newBase = mathutil.MaxInt64(currentEnd, requiredBase) - newEnd = mathutil.MinInt64(math.MaxInt64-step, newBase) + step + newEnd = mathutil.MinInt64(math.MaxInt64-alloc.step, newBase) + alloc.step } else { if currentEnd >= requiredBase { newBase = currentEnd @@ -215,6 +224,8 @@ func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { if alloc.base == alloc.end { // step var newBase, newEnd int64 startTime := time.Now() + consumeDur := startTime.Sub(alloc.lastAllocTime) + alloc.step = NextStep(alloc.step, consumeDur) err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -222,7 +233,7 @@ func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { if err1 != nil { return err1 } - tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(step))) + tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(alloc.step))) newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) return err1 }) @@ -230,6 +241,7 @@ func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { if err != nil { return 0, err } + alloc.lastAllocTime = time.Now() if uint64(newBase) == math.MaxUint64 { return 0, ErrAutoincReadFailed } @@ -248,6 +260,8 @@ func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { if alloc.base == alloc.end { // step var newBase, newEnd int64 startTime := time.Now() + consumeDur := startTime.Sub(alloc.lastAllocTime) + alloc.step = NextStep(alloc.step, consumeDur) err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -255,7 +269,7 @@ func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { if err1 != nil { return err1 } - tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, step) + tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, alloc.step) newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) return err1 }) @@ -263,6 +277,7 @@ func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { if err != nil { return 0, err } + alloc.lastAllocTime = time.Now() if newBase == math.MaxInt64 { return 0, ErrAutoincReadFailed } @@ -290,12 +305,32 @@ func (alloc *allocator) Alloc(tableID int64) (int64, error) { return alloc.alloc4Signed(tableID) } +// NextStep return new auto id step according to previous step and consuming time. +func NextStep(curStep int64, consumeDur time.Duration) int64 { + failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(step) + } + }) + + consumeRate := defaultConsumeTime.Seconds() / consumeDur.Seconds() + res := int64(float64(curStep) * consumeRate) + if res < minStep { + return minStep + } else if res > maxStep { + return maxStep + } + return res +} + // NewAllocator returns a new auto increment id generator on the store. func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator { return &allocator{ - store: store, - dbID: dbID, - isUnsigned: isUnsigned, + store: store, + dbID: dbID, + isUnsigned: isUnsigned, + step: step, + lastAllocTime: time.Now(), } } diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 569578b7c4bbd..6714e322b4076 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -21,6 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -39,6 +40,11 @@ type testSuite struct { } func (*testSuite) TestT(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() + store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) defer store.Close() @@ -130,6 +136,11 @@ func (*testSuite) TestT(c *C) { } func (*testSuite) TestUnsignedAutoid(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + }() + store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) defer store.Close() @@ -315,3 +326,13 @@ func (*testSuite) TestRollbackAlloc(c *C) { c.Assert(alloc.Base(), Equals, int64(0)) c.Assert(alloc.End(), Equals, int64(0)) } + +// TestNextStep tests generate next auto id step. +func (*testSuite) TestNextStep(c *C) { + nextStep := autoid.NextStep(2000000, 1*time.Nanosecond) + c.Assert(nextStep, Equals, int64(2000000)) + nextStep = autoid.NextStep(678910, 10*time.Second) + c.Assert(nextStep, Equals, int64(678910)) + nextStep = autoid.NextStep(50000, 10*time.Minute) + c.Assert(nextStep, Equals, int64(1000)) +} From 86a6f9b15571ccd36ee5be47995931a8946b9d32 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 1 Jul 2019 21:33:08 +0800 Subject: [PATCH 05/21] *: fix oom action cancel bug (#10993) (#11004) --- executor/executor_test.go | 19 +++++++++++++++++++ store/tikv/coprocessor.go | 2 +- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index a5fe033ecfdb1..2b5aaa145f229 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -60,6 +60,7 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" @@ -3958,6 +3959,24 @@ func (s *testOOMSuite) TestDistSQLMemoryControl(c *C) { tk.Se.GetSessionVars().MemQuotaDistSQL = -1 } +func (s *testSuite) TestOOMPanicAction(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int primary key, b double);") + tk.MustExec("insert into t values (1,1)") + sm := &mockSessionManager1{ + PS: make([]*util.ProcessInfo, 0), + } + tk.Se.SetSessionManager(sm) + s.domain.ExpensiveQueryHandle().SetSessionManager(sm) + config.GetGlobalConfig().OOMAction = config.OOMActionCancel + tk.MustExec("set @@tidb_mem_quota_query=1;") + err := tk.QueryToErr("select sum(b) from t group by a;") + c.Assert(err, NotNil) + c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*") +} + type oomCapturer struct { zapcore.Core tracker string diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index a0c7e37eb084e..251ec6796e8d6 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -620,7 +620,7 @@ func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh zap.Stack("stack trace")) resp := &copResponse{err: errors.Errorf("%v", r)} // if panic has happened, set checkOOM to false to avoid another panic. - worker.sendToRespCh(resp, task.respChan, false) + worker.sendToRespCh(resp, respCh, false) } }() remainTasks := []*copTask{task} From 6391d58d25124b1506c237b0a9bf44fd3bb67fa3 Mon Sep 17 00:00:00 2001 From: Feng Liyuan Date: Mon, 1 Jul 2019 21:45:09 +0800 Subject: [PATCH 06/21] =?UTF-8?q?distsql:=20clean=20the=20memory=20usage?= =?UTF-8?q?=20of=20MemTracker=20when=20a=20query=20end=E2=80=A6=20(#10970)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- distsql/distsql_test.go | 17 +++++++++++++ distsql/request_builder_test.go | 5 ++++ distsql/select_result.go | 45 +++++++++++++++++++++++---------- 3 files changed, 53 insertions(+), 14 deletions(-) diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 478881e47720f..6fda7832eb474 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -42,6 +42,7 @@ func (s *testSuite) createSelectNormal(batch, totalRows int, c *C, planIDs []str SetDesc(false). SetKeepOrder(false). SetFromSessionVars(variable.NewSessionVars()). + SetMemTracker(s.sctx, stringutil.StringerStr("testSuite.createSelectNormal")). Build() c.Assert(err, IsNil) @@ -106,6 +107,21 @@ func (s *testSuite) TestSelectNormal(c *C) { c.Assert(numAllRows, Equals, 2) err := response.Close() c.Assert(err, IsNil) + c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0)) +} + +func (s *testSuite) TestSelectMemTracker(c *C) { + response, colTypes := s.createSelectNormal(2, 6, c, nil) + response.Fetch(context.TODO()) + + // Test Next. + chk := chunk.New(colTypes, 3, 3) + err := response.Next(context.TODO(), chk) + c.Assert(err, IsNil) + c.Assert(chk.IsFull(), Equals, true) + err = response.Close() + c.Assert(err, IsNil) + c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0)) } func (s *testSuite) TestSelectNormalChunkSize(c *C) { @@ -113,6 +129,7 @@ func (s *testSuite) TestSelectNormalChunkSize(c *C) { response.Fetch(context.TODO()) s.testChunkSize(response, colTypes, c) c.Assert(response.Close(), IsNil) + c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0)) } func (s *testSuite) TestSelectWithRuntimeStats(c *C) { diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index b64dd63218892..a2b472b5ad833 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -26,8 +26,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tipb/go-tipb" ) @@ -49,6 +51,9 @@ type testSuite struct { func (s *testSuite) SetUpSuite(c *C) { ctx := mock.NewContext() + ctx.GetSessionVars().StmtCtx = &stmtctx.StatementContext{ + MemTracker: memory.NewTracker(stringutil.StringerStr("testSuite"), variable.DefTiDBMemQuotaDistSQL), + } ctx.Store = &mock.Store{ Client: &mock.Client{ MockResponse: &mockResponse{ diff --git a/distsql/select_result.go b/distsql/select_result.go index 72a283d1b2f8f..3935df52ad450 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -66,8 +66,9 @@ type selectResult struct { fieldTypes []*types.FieldType ctx sessionctx.Context - selectResp *tipb.SelectResponse - respChkIdx int + selectResp *tipb.SelectResponse + selectRespSize int // record the selectResp.Size() when it is initialized. + respChkIdx int feedback *statistics.QueryFeedback partialCount int64 // number of partial results. @@ -103,20 +104,25 @@ func (r *selectResult) fetch(ctx context.Context) { if err != nil { result.err = err } else if resultSubset == nil { + // If the result is drained, the resultSubset would be nil return } else { result.result = resultSubset - if r.memTracker != nil { - r.memTracker.Consume(int64(resultSubset.MemSize())) - } + r.memConsume(int64(resultSubset.MemSize())) } select { case r.results <- result: case <-r.closed: // If selectResult called Close() already, make fetch goroutine exit. + if resultSubset != nil { + r.memConsume(-int64(resultSubset.MemSize())) + } return case <-ctx.Done(): + if resultSubset != nil { + r.memConsume(-int64(resultSubset.MemSize())) + } return } } @@ -161,24 +167,21 @@ func (r *selectResult) getSelectResp() error { if re.err != nil { return errors.Trace(re.err) } - if r.memTracker != nil && r.selectResp != nil { - r.memTracker.Consume(-int64(r.selectResp.Size())) + if r.selectResp != nil { + r.memConsume(-int64(r.selectRespSize)) } if re.result == nil { r.selectResp = nil return nil } - if r.memTracker != nil { - r.memTracker.Consume(-int64(re.result.MemSize())) - } + r.memConsume(-int64(re.result.MemSize())) r.selectResp = new(tipb.SelectResponse) err := r.selectResp.Unmarshal(re.result.GetData()) if err != nil { return errors.Trace(err) } - if r.memTracker != nil && r.selectResp != nil { - r.memTracker.Consume(int64(r.selectResp.Size())) - } + r.selectRespSize = r.selectResp.Size() + r.memConsume(int64(r.selectRespSize)) if err := r.selectResp.Error; err != nil { return terror.ClassTiKV.New(terror.ErrCode(err.Code), err.Msg) } @@ -234,13 +237,27 @@ func (r *selectResult) readRowsData(chk *chunk.Chunk) (err error) { return nil } +func (r *selectResult) memConsume(bytes int64) { + if r.memTracker != nil { + r.memTracker.Consume(bytes) + } +} + // Close closes selectResult. func (r *selectResult) Close() error { - // Close this channel tell fetch goroutine to exit. if r.feedback.Actual() >= 0 { metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual())) } metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount)) + // Close this channel to tell the fetch goroutine to exit. close(r.closed) + for re := range r.results { + if re.result != nil { + r.memConsume(-int64(re.result.MemSize())) + } + } + if r.selectResp != nil { + r.memConsume(-int64(r.selectRespSize)) + } return r.resp.Close() } From f865967de078997951b74c2daa36a0401073dfc4 Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 2 Jul 2019 11:08:22 +0800 Subject: [PATCH 07/21] plugin: setup connection info in session when audit plugin be enabled (#10923) (#11013) --- plugin/plugin.go | 15 +++++++++++++++ server/conn.go | 6 +++++- server/server.go | 13 ++++++++----- sessionctx/variable/session.go | 3 +++ 4 files changed, 31 insertions(+), 6 deletions(-) diff --git a/plugin/plugin.go b/plugin/plugin.go index ee5a7d68c6de6..a7c61cbd5819c 100644 --- a/plugin/plugin.go +++ b/plugin/plugin.go @@ -377,6 +377,21 @@ func ForeachPlugin(kind Kind, fn func(plugin *Plugin) error) error { return nil } +// IsEnable checks plugin's enable state. +func IsEnable(kind Kind) bool { + plugins := pluginGlobal.plugins() + if plugins == nil { + return false + } + for i := range plugins.plugins[kind] { + p := &plugins.plugins[kind][i] + if p.State == Ready { + return true + } + } + return false +} + // GetAll finds and returns all plugins. func GetAll() map[Kind][]Plugin { plugins := pluginGlobal.plugins() diff --git a/server/conn.go b/server/conn.go index 8f2b99598a30c..7a56f1f799f8f 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1434,10 +1434,14 @@ func (cc *clientConn) handleChangeUser(ctx context.Context, data []byte) error { return err } + if plugin.IsEnable(plugin.Audit) { + cc.ctx.GetSessionVars().ConnectionInfo = cc.connectInfo() + } + err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { authPlugin := plugin.DeclareAuditManifest(p.Manifest) if authPlugin.OnConnectionEvent != nil { - connInfo := cc.connectInfo() + connInfo := cc.ctx.GetSessionVars().ConnectionInfo err = authPlugin.OnConnectionEvent(context.Background(), &auth.UserIdentity{Hostname: connInfo.Host}, plugin.ChangeUser, connInfo) if err != nil { return err diff --git a/server/server.go b/server/server.go index c258f074d0ed0..dfc608682023a 100644 --- a/server/server.go +++ b/server/server.go @@ -422,11 +422,14 @@ func (s *Server) onConn(conn *clientConn) { s.rwlock.Unlock() metrics.ConnGauge.Set(float64(connections)) + if plugin.IsEnable(plugin.Audit) { + conn.ctx.GetSessionVars().ConnectionInfo = conn.connectInfo() + } err := plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { authPlugin := plugin.DeclareAuditManifest(p.Manifest) if authPlugin.OnConnectionEvent != nil { - connInfo := conn.connectInfo() - return authPlugin.OnConnectionEvent(context.Background(), conn.ctx.GetSessionVars().User, plugin.Connected, connInfo) + sessionVars := conn.ctx.GetSessionVars() + return authPlugin.OnConnectionEvent(context.Background(), sessionVars.User, plugin.Connected, sessionVars.ConnectionInfo) } return nil }) @@ -440,9 +443,9 @@ func (s *Server) onConn(conn *clientConn) { err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { authPlugin := plugin.DeclareAuditManifest(p.Manifest) if authPlugin.OnConnectionEvent != nil { - connInfo := conn.connectInfo() - connInfo.Duration = float64(time.Since(connectedTime)) / float64(time.Millisecond) - err := authPlugin.OnConnectionEvent(context.Background(), conn.ctx.GetSessionVars().User, plugin.Disconnect, connInfo) + sessionVars := conn.ctx.GetSessionVars() + sessionVars.ConnectionInfo.Duration = float64(time.Since(connectedTime)) / float64(time.Millisecond) + err := authPlugin.OnConnectionEvent(context.Background(), sessionVars.User, plugin.Disconnect, sessionVars.ConnectionInfo) if err != nil { logutil.Logger(context.Background()).Warn("do connection event failed", zap.String("plugin", authPlugin.Name), zap.Error(err)) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 83d060a814f93..045445b9300aa 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -382,6 +382,9 @@ type SessionVars struct { // Killed is a flag to indicate that this query is killed. Killed uint32 + + // ConnectionInfo indicates current connection info used by current session, only be lazy assigned by plugin. + ConnectionInfo *ConnectionInfo } // ConnectionInfo present connection used by audit. From e04723a56fe84785145e1e88eb3befcacaf3b86f Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 2 Jul 2019 11:44:14 +0800 Subject: [PATCH 08/21] ddl: add recover for run ddl job (#10981) (#11022) --- ddl/ddl_worker.go | 14 +++++++++++++- ddl/failtest/fail_db_test.go | 14 ++++++++++++++ ddl/table.go | 2 +- infoschema/infoschema.go | 5 +++++ 4 files changed, 33 insertions(+), 2 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 2151d9efebb78..6ec289f3ebd82 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl/util" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -397,7 +399,14 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { // If running job meets error, we will save this error in job Error // and retry later if the job is not cancelled. - schemaVer, runJobErr = w.runDDLJob(d, t, job) + tidbutil.WithRecovery(func() { + schemaVer, runJobErr = w.runDDLJob(d, t, job) + }, func(r interface{}) { + if r != nil { + // If run ddl job panic, just cancel the ddl jobs. + job.State = model.JobStateCancelling + } + }) if job.IsCancelled() { txn.Reset() err = w.finishDDLJob(t, job) @@ -469,6 +478,9 @@ func chooseLeaseTime(t, max time.Duration) time.Duration { // runDDLJob runs a DDL job. It returns the current schema version in this transaction and the error. func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + // Mock for run ddl job panic. + failpoint.Inject("mockPanicInRunDDLJob", func(_ failpoint.Value) {}) + logutil.Logger(w.logCtx).Info("[ddl] run DDL job", zap.String("job", job.String())) timeStart := time.Now() defer func() { diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index a8ad59c218ed0..8488cda6cddf7 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -386,3 +386,17 @@ LOOP: tk.MustExec("admin check table test_add_index") tk.MustExec("drop table test_add_index") } + +// TestRunDDLJobPanic tests recover panic when run ddl job panic. +func (s *testFailDBSuite) TestRunDDLJobPanic(c *C) { + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockPanicInRunDDLJob"), IsNil) + }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockPanicInRunDDLJob", `1*panic("panic test")`), IsNil) + _, err := tk.Exec("create table t(c1 int, c2 int)") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job") +} diff --git a/ddl/table.go b/ddl/table.go index a466f40e2f2af..3232fa3ab8ae2 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -765,7 +765,7 @@ func onModifyTableCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _ func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) error { // d.infoHandle maybe nil in some test. - if d.infoHandle == nil { + if d.infoHandle == nil || !d.infoHandle.IsValid() { return checkTableNotExistsFromStore(t, schemaID, tableName) } // Try to use memory schema info to check first. diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index ed455b3dacfcb..7b0b5f612cae0 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -292,6 +292,11 @@ func (h *Handle) Get() InfoSchema { return schema } +// IsValid uses to check whether handle value is valid. +func (h *Handle) IsValid() bool { + return h.value.Load() != nil +} + // EmptyClone creates a new Handle with the same store and memSchema, but the value is not set. func (h *Handle) EmptyClone() *Handle { newHandle := &Handle{ From 6b5fcf7ec51505a9fbd9a2633cc342689208a8a9 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Tue, 2 Jul 2019 13:35:36 +0800 Subject: [PATCH 09/21] executor: fix point get snapshot TS for pessimistic transaction. (#11012) (#11015) --- executor/point_get.go | 6 +++++- session/pessimistic_test.go | 29 +++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/executor/point_get.go b/executor/point_get.go index 80730ca2c48c4..955d68ede1032 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -82,8 +82,12 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { return nil } e.done = true + snapshotTS := e.startTS + if e.lock { + snapshotTS = e.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + } var err error - e.snapshot, err = e.ctx.GetStore().GetSnapshot(kv.Version{Ver: e.startTS}) + e.snapshot, err = e.ctx.GetStore().GetSnapshot(kv.Version{Ver: snapshotTS}) if err != nil { return err } diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 1fb6332b1c8b6..b8110e4f5d488 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -322,3 +322,32 @@ func (s *testPessimisticSuite) TestPointGetKeyLock(c *C) { tk.MustExec("commit") syncCh <- struct{}{} } + +func (s *testPessimisticSuite) TestBankTransfer(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists accounts") + tk.MustExec("create table accounts (id int primary key, c int)") + tk.MustExec("insert accounts values (1, 100), (2, 100), (3, 100)") + syncCh := make(chan struct{}) + + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from accounts where id = 1 for update").Check(testkit.Rows("1 100")) + go func() { + tk2.MustExec("begin pessimistic") + tk2.MustExec("select * from accounts where id = 2 for update") + <-syncCh + tk2.MustExec("select * from accounts where id = 3 for update") + tk2.MustExec("update accounts set c = 50 where id = 2") + tk2.MustExec("update accounts set c = 150 where id = 3") + tk2.MustExec("commit") + <-syncCh + }() + syncCh <- struct{}{} + tk.MustQuery("select * from accounts where id = 2 for update").Check(testkit.Rows("2 50")) + tk.MustExec("update accounts set c = 50 where id = 1") + tk.MustExec("update accounts set c = 100 where id = 2") + tk.MustExec("commit") + syncCh <- struct{}{} + tk.MustQuery("select sum(c) from accounts").Check(testkit.Rows("300")) +} From 2e406b011fd80ecf53e44f710834ca241817c03e Mon Sep 17 00:00:00 2001 From: bb7133 Date: Tue, 2 Jul 2019 23:34:47 +0800 Subject: [PATCH 10/21] planner: fix a corner case in column pruning rule (#10974) (#11020) --- cmd/explaintest/r/explain_easy.result | 29 +++++++++++++++++++++++++++ cmd/explaintest/t/explain_easy.test | 6 ++++++ planner/core/logical_plan_test.go | 18 +++++++++++++++++ planner/core/rule_column_pruning.go | 17 ++++++++++++++++ 4 files changed, 70 insertions(+) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 9ca1062c69484..0d2d9faae241a 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -192,6 +192,35 @@ HashAgg_18 24000.00 root group by:c1, funcs:firstrow(join_agg_0) └─IndexReader_62 8000.00 root index:StreamAgg_53 └─StreamAgg_53 8000.00 cop group by:test.t2.c1, funcs:firstrow(test.t2.c1), firstrow(test.t2.c1) └─IndexScan_60 10000.00 cop table:t2, index:c1, range:[NULL,+inf], keep order:true, stats:pseudo +explain select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; +id count task operator info +StreamAgg_13 1.00 root funcs:count(1) +└─StreamAgg_28 1.00 root funcs:firstrow(col_0) + └─TableReader_29 1.00 root data:StreamAgg_17 + └─StreamAgg_17 1.00 cop funcs:firstrow(1) + └─Selection_27 10.00 cop eq(test.t1.c3, 100) + └─TableScan_26 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +explain select 1 from (select count(c2), count(c3) from t1) k; +id count task operator info +Projection_5 1.00 root 1 +└─StreamAgg_17 1.00 root funcs:firstrow(col_0) + └─TableReader_18 1.00 root data:StreamAgg_9 + └─StreamAgg_9 1.00 cop funcs:firstrow(1) + └─TableScan_16 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +explain select count(1) from (select max(c2), count(c3) as m from t1) k; +id count task operator info +StreamAgg_11 1.00 root funcs:count(1) +└─StreamAgg_23 1.00 root funcs:firstrow(col_0) + └─TableReader_24 1.00 root data:StreamAgg_15 + └─StreamAgg_15 1.00 cop funcs:firstrow(1) + └─TableScan_22 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo +explain select count(1) from (select count(c2) from t1 group by c3) k; +id count task operator info +StreamAgg_11 1.00 root funcs:count(1) +└─HashAgg_23 8000.00 root group by:col_1, funcs:firstrow(col_0) + └─TableReader_24 8000.00 root data:HashAgg_20 + └─HashAgg_20 8000.00 cop group by:test.t1.c3, funcs:firstrow(1) + └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo set @@session.tidb_opt_insubq_to_join_and_agg=0; explain select sum(t1.c1 in (select c1 from t2)) from t1; id count task operator info diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index c5044c96069cd..fe056d84d0208 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -35,6 +35,12 @@ explain select if(10, t1.c1, t1.c2) from t1; explain select c1 from t2 union select c1 from t2 union all select c1 from t2; explain select c1 from t2 union all select c1 from t2 union select c1 from t2; +# https://github.com/pingcap/tidb/issues/9125 +explain select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; +explain select 1 from (select count(c2), count(c3) from t1) k; +explain select count(1) from (select max(c2), count(c3) as m from t1) k; +explain select count(1) from (select count(c2) from t1 group by c3) k; + set @@session.tidb_opt_insubq_to_join_and_agg=0; explain select sum(t1.c1 in (select c1 from t2)) from t1; diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 71598d87d2203..702a3f7d1a5d4 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1148,6 +1148,24 @@ func (s *testPlanSuite) TestColumnPruning(c *C) { 12: {"test.t4.a"}, }, }, + { + sql: "select 1 from (select count(b) as cnt from t) t1;", + ans: map[int][]string{ + 1: {"test.t.a"}, + }, + }, + { + sql: "select count(1) from (select count(b) as cnt from t) t1;", + ans: map[int][]string{ + 1: {"test.t.a"}, + }, + }, + { + sql: "select count(1) from (select count(b) as cnt from t group by c) t1;", + ans: map[int][]string{ + 1: {"test.t.c"}, + }, + }, } for _, tt := range tests { comment := Commentf("for %s", tt.sql) diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 1a78a8ecb4d90..6a5a278921388 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -20,7 +20,9 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/types" ) type columnPruner struct { @@ -117,6 +119,21 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) for _, aggrFunc := range la.AggFuncs { selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, aggrFunc.Args, nil) } + if len(la.AggFuncs) == 0 { + // If all the aggregate functions are pruned, we should add an aggregate function to keep the correctness. + one, err := aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.One}, false) + if err != nil { + return err + } + la.AggFuncs = []*aggregation.AggFuncDesc{one} + col := &expression.Column{ + ColName: model.NewCIStr("dummy_agg"), + UniqueID: la.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: types.NewFieldType(mysql.TypeLonglong), + } + la.schema.Columns = []*expression.Column{col} + } + if len(la.GroupByItems) > 0 { for i := len(la.GroupByItems) - 1; i >= 0; i-- { cols := expression.ExtractColumns(la.GroupByItems[i]) From 0d7ac5f4d51d64b1ad54d34cb7d934430ff56d66 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 3 Jul 2019 11:04:04 +0800 Subject: [PATCH 11/21] =?UTF-8?q?expression:=20make=20`sleep`=20function?= =?UTF-8?q?=20response=20to=20the=20kill=20stateme=E2=80=A6=20(#11028)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- expression/builtin_miscellaneous.go | 22 +++++++++++++++---- expression/evaluator_test.go | 33 ++++++++++++++--------------- 2 files changed, 34 insertions(+), 21 deletions(-) diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index 9308b73e3ea02..239489acd2272 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -19,6 +19,7 @@ import ( "math" "net" "strings" + "sync/atomic" "time" "github.com/google/uuid" @@ -131,12 +132,25 @@ func (b *builtinSleepSig) evalInt(row chunk.Row) (int64, bool, error) { if val > math.MaxFloat64/float64(time.Second.Nanoseconds()) { return 0, false, errIncorrectArgs.GenWithStackByArgs("sleep") } + dur := time.Duration(val * float64(time.Second.Nanoseconds())) - select { - case <-time.After(dur): - // TODO: Handle Ctrl-C is pressed in `mysql` client. - // return 1 when SLEEP() is KILLed + ticker := time.NewTicker(10 * time.Millisecond) + defer ticker.Stop() + start := time.Now() + finish := false + for !finish { + select { + case now := <-ticker.C: + if now.Sub(start) > dur { + finish = true + } + default: + if atomic.CompareAndSwapUint32(&sessVars.Killed, 1, 0) { + return 1, false, nil + } + } } + return 0, false, nil } diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index 8c04dd8bb59ac..db1bbd54d02b5 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -14,6 +14,7 @@ package expression import ( + "sync/atomic" "testing" "time" @@ -176,23 +177,21 @@ func (s *testEvaluatorSuite) TestSleep(c *C) { sub := time.Since(start) c.Assert(sub.Nanoseconds(), GreaterEqual, int64(0.5*1e9)) - // quit when context canceled. - // TODO: recover it. - // d[0].SetFloat64(2) - // f, err = fc.getFunction(ctx, s.datumsToConstants(d)) - // c.Assert(err, IsNil) - // start = time.Now() - // go func() { - // time.Sleep(1 * time.Second) - // ctx.Cancel() - // }() - // ret, isNull, err = f.evalInt(chunk.Row{}) - // sub = time.Since(start) - // c.Assert(err, IsNil) - // c.Assert(isNull, IsFalse) - // c.Assert(ret, Equals, int64(1)) - // c.Assert(sub.Nanoseconds(), LessEqual, int64(2*1e9)) - // c.Assert(sub.Nanoseconds(), GreaterEqual, int64(1*1e9)) + d[0].SetFloat64(3) + f, err = fc.getFunction(ctx, s.datumsToConstants(d)) + c.Assert(err, IsNil) + start = time.Now() + go func() { + time.Sleep(1 * time.Second) + atomic.CompareAndSwapUint32(&ctx.GetSessionVars().Killed, 0, 1) + }() + ret, isNull, err = f.evalInt(chunk.Row{}) + sub = time.Since(start) + c.Assert(err, IsNil) + c.Assert(isNull, IsFalse) + c.Assert(ret, Equals, int64(1)) + c.Assert(sub.Nanoseconds(), LessEqual, int64(2*1e9)) + c.Assert(sub.Nanoseconds(), GreaterEqual, int64(1*1e9)) } func (s *testEvaluatorSuite) TestBinopComparison(c *C) { From 3cc902159f7d12da6726420bbd039084491f1db4 Mon Sep 17 00:00:00 2001 From: wshwsh12 <793703860@qq.com> Date: Wed, 3 Jul 2019 11:18:12 +0800 Subject: [PATCH 12/21] *: fix 'db' and 'Info' column of 'show processlist' (#10985) (#11003) --- infoschema/tables.go | 2 +- infoschema/tables_test.go | 47 +++++++++++++++++++++++++++ session/session.go | 13 ++++++-- util/expensivequery/expensivequery.go | 11 ++++--- util/processinfo.go | 16 +++++---- 5 files changed, 75 insertions(+), 14 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 4a64164f69127..2ec17dc7f7b75 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -538,7 +538,7 @@ var tableProcesslistCols = []columnInfo{ {"ID", mysql.TypeLonglong, 21, mysql.NotNullFlag, 0, nil}, {"USER", mysql.TypeVarchar, 16, mysql.NotNullFlag, "", nil}, {"HOST", mysql.TypeVarchar, 64, mysql.NotNullFlag, "", nil}, - {"DB", mysql.TypeVarchar, 64, mysql.NotNullFlag, "", nil}, + {"DB", mysql.TypeVarchar, 64, 0, nil, nil}, {"COMMAND", mysql.TypeVarchar, 16, mysql.NotNullFlag, "", nil}, {"TIME", mysql.TypeLong, 7, mysql.NotNullFlag, 0, nil}, {"STATE", mysql.TypeVarchar, 7, 0, nil, nil}, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6788d585656d3..057ec4082ed25 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -17,6 +17,7 @@ import ( "fmt" "os" "strconv" + "strings" . "github.com/pingcap/check" "github.com/pingcap/parser/auth" @@ -288,6 +289,52 @@ func (s *testTableSuite) TestSomeTables(c *C) { tk.MustQuery("select * from information_schema.PROCESSLIST order by ID;").Check( testkit.Rows("1 user-1 localhost information_schema Quit 9223372036 1 do something", "2 user-2 localhost test Init DB 9223372036 2 do something")) + + sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm.processInfoMap[1] = &util.ProcessInfo{ + ID: 1, + User: "user-1", + Host: "localhost", + DB: "information_schema", + Command: byte(1), + State: 1, + Info: nil, + StmtCtx: tk.Se.GetSessionVars().StmtCtx, + } + sm.processInfoMap[2] = &util.ProcessInfo{ + ID: 2, + User: "user-2", + Host: "localhost", + DB: nil, + Command: byte(2), + State: 2, + Info: strings.Repeat("x", 101), + StmtCtx: tk.Se.GetSessionVars().StmtCtx, + } + tk.Se.SetSessionManager(sm) + tk.MustQuery("select * from information_schema.PROCESSLIST order by ID;").Check( + testkit.Rows( + fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s", ""), + fmt.Sprintf("2 user-2 localhost Init DB 9223372036 2 %s", strings.Repeat("x", 101)), + )) + tk.MustQuery("SHOW PROCESSLIST;").Sort().Check( + testkit.Rows( + fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s", ""), + fmt.Sprintf("2 user-2 localhost Init DB 9223372036 2 %s", strings.Repeat("x", 100)), + )) + tk.MustQuery("SHOW FULL PROCESSLIST;").Sort().Check( + testkit.Rows( + fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s", ""), + fmt.Sprintf("2 user-2 localhost Init DB 9223372036 2 %s", strings.Repeat("x", 101)), + )) + tk.MustQuery("select * from information_schema.PROCESSLIST where db is null;").Check( + testkit.Rows( + fmt.Sprintf("2 user-2 localhost Init DB 9223372036 2 %s", strings.Repeat("x", 101)), + )) + tk.MustQuery("select * from information_schema.PROCESSLIST where Info is null;").Check( + testkit.Rows( + fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s", ""), + )) } func (s *testTableSuite) TestSchemataCharacterSet(c *C) { diff --git a/session/session.go b/session/session.go index 38c95182a3046..676ae9bc72410 100644 --- a/session/session.go +++ b/session/session.go @@ -908,14 +908,23 @@ func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) } func (s *session) SetProcessInfo(sql string, t time.Time, command byte) { + var db interface{} + if len(s.sessionVars.CurrentDB) > 0 { + db = s.sessionVars.CurrentDB + } + + var info interface{} + if len(sql) > 0 { + info = sql + } pi := util.ProcessInfo{ ID: s.sessionVars.ConnectionID, - DB: s.sessionVars.CurrentDB, + DB: db, Command: command, Plan: s.currentPlan, Time: t, State: s.Status(), - Info: sql, + Info: info, CurTxnStartTS: s.sessionVars.TxnCtx.StartTS, StmtCtx: s.sessionVars.StmtCtx, StatsInfo: plannercore.GetStatsInfo, diff --git a/util/expensivequery/expensivequery.go b/util/expensivequery/expensivequery.go index 2689eac6092a8..1b8df18ad69f5 100644 --- a/util/expensivequery/expensivequery.go +++ b/util/expensivequery/expensivequery.go @@ -60,7 +60,7 @@ func (eqh *Handle) Run() { } processInfo := eqh.sm.ShowProcessList() for _, info := range processInfo { - if len(info.Info) == 0 || info.ExceedExpensiveTimeThresh { + if info.Info == nil || info.ExceedExpensiveTimeThresh { continue } if costTime := time.Since(info.Time); costTime >= curInterval { @@ -126,8 +126,8 @@ func logExpensiveQuery(costTime time.Duration, info *util.ProcessInfo) { if len(info.User) > 0 { logFields = append(logFields, zap.String("user", info.User)) } - if len(info.DB) > 0 { - logFields = append(logFields, zap.String("database", info.DB)) + if info.DB != nil && len(info.DB.(string)) > 0 { + logFields = append(logFields, zap.String("database", info.DB.(string))) } var tableIDs, indexIDs string if len(info.StmtCtx.TableIDs) > 0 { @@ -144,7 +144,10 @@ func logExpensiveQuery(costTime time.Duration, info *util.ProcessInfo) { } const logSQLLen = 1024 * 8 - sql := info.Info + var sql string + if info.Info != nil { + sql = info.Info.(string) + } if len(sql) > logSQLLen { sql = fmt.Sprintf("%s len(%d)", sql[:logSQLLen], len(sql)) } diff --git a/util/processinfo.go b/util/processinfo.go index 956196cdfdb42..6d0fab3e59eb7 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -26,12 +26,12 @@ type ProcessInfo struct { ID uint64 User string Host string - DB string + DB interface{} Command byte Plan interface{} Time time.Time State uint16 - Info string + Info interface{} CurTxnStartTS uint64 StmtCtx *stmtctx.StatementContext StatsInfo func(interface{}) map[string]uint64 @@ -40,11 +40,13 @@ type ProcessInfo struct { // ToRow returns []interface{} for the row data of "show processlist" and "select * from infoschema.processlist". func (pi *ProcessInfo) ToRow(full bool) []interface{} { - var info string - if full { - info = pi.Info - } else { - info = fmt.Sprintf("%.100v", pi.Info) + var info interface{} + if pi.Info != nil { + if full { + info = pi.Info.(string) + } else { + info = fmt.Sprintf("%.100v", pi.Info.(string)) + } } t := uint64(time.Since(pi.Time) / time.Second) return []interface{}{ From 3acb6459174f1829857698b8e565d0ab337cc6ba Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 3 Jul 2019 16:07:13 +0800 Subject: [PATCH 13/21] planner: make WindowFunction uncacheable in prepare (#11035) (#11048) --- planner/core/cacheable_checker.go | 5 +++++ planner/core/cacheable_checker_test.go | 3 +++ 2 files changed, 8 insertions(+) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 49e08eb8227b1..f4ef4f9c22b6d 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -82,6 +82,11 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren return in, true } } + case *ast.FrameBound: + if _, ok := node.Expr.(*driver.ParamMarkerExpr); ok { + checker.cacheable = false + return in, true + } } return in, false } diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index 8f3d287701533..6d195c25f4b27 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -191,4 +191,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { OrderBy: orderByClause, } c.Assert(Cacheable(stmt), IsTrue) + + boundExpr := &ast.FrameBound{Expr: &driver.ParamMarkerExpr{}} + c.Assert(Cacheable(boundExpr), IsFalse) } From aafe172b62ab68d3e6273edced2f154bb21ba141 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Wed, 3 Jul 2019 17:21:57 +0800 Subject: [PATCH 14/21] store/tikv: revert optimization for optimistic conflicts pessimistic (#11051) --- session/pessimistic_test.go | 33 ++++++++++++++++++++++++++++++++- store/tikv/2pc.go | 13 ------------- store/tikv/2pc_test.go | 17 ----------------- 3 files changed, 32 insertions(+), 31 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index b8110e4f5d488..a77e3a74c3f31 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -46,6 +46,8 @@ type testPessimisticSuite struct { func (s *testPessimisticSuite) SetUpSuite(c *C) { testleak.BeforeTest() config.GetGlobalConfig().PessimisticTxn.Enable = true + // Set it to 300ms for testing lock resolve. + tikv.PessimisticLockTTL = 300 s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) s.mvccStore = mocktikv.MustNewMVCCStore() @@ -53,7 +55,6 @@ func (s *testPessimisticSuite) SetUpSuite(c *C) { mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), ) - tikv.PessimisticLockTTL = uint64(config.MinPessimisticTTL / time.Millisecond) c.Assert(err, IsNil) s.store = store session.SetSchemaLease(0) @@ -351,3 +352,33 @@ func (s *testPessimisticSuite) TestBankTransfer(c *C) { syncCh <- struct{}{} tk.MustQuery("select sum(c) from accounts").Check(testkit.Rows("300")) } + +func (s *testPessimisticSuite) TestOptimisticConflicts(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists conflict") + tk.MustExec("create table conflict (id int primary key, c int)") + tk.MustExec("insert conflict values (1, 1)") + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from conflict where id = 1 for update") + syncCh := make(chan struct{}) + go func() { + tk2.MustExec("update conflict set c = 3 where id = 1") + <-syncCh + }() + time.Sleep(time.Millisecond * 10) + tk.MustExec("update conflict set c = 2 where id = 1") + tk.MustExec("commit") + syncCh <- struct{}{} + tk.MustQuery("select c from conflict where id = 1").Check(testkit.Rows("3")) + + // Check outdated pessimistic lock is resolved. + tk.MustExec("begin pessimistic") + tk.MustExec("update conflict set c = 4 where id = 1") + time.Sleep(300 * time.Millisecond) + tk2.MustExec("begin optimistic") + tk2.MustExec("update conflict set c = 5 where id = 1") + tk2.MustExec("commit") + _, err := tk.Exec("commit") + c.Check(err, NotNil) +} diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 705da738ac03a..6b888d4c162fd 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -538,19 +538,6 @@ func (c *twoPhaseCommitter) prewriteSingleBatch(bo *Backoffer, batch batchKeys) if err1 != nil { return errors.Trace(err1) } - if !c.isPessimistic && c.lockTTL < lock.TTL && lock.TTL >= uint64(config.MinPessimisticTTL/time.Millisecond) { - // An optimistic prewrite meets a pessimistic or large transaction lock. - // If we wait for the lock, other written optimistic locks would block reads for long time. - // And it is very unlikely this transaction would succeed after wait for the long TTL lock. - // Return write conflict error to cleanup locks. - return newWriteConflictError(&pb.WriteConflict{ - StartTs: c.startTS, - ConflictTs: lock.TxnID, - ConflictCommitTs: 0, - Key: lock.Key, - Primary: lock.Primary, - }) - } logutil.Logger(context.Background()).Debug("prewrite encounters lock", zap.Uint64("conn", c.connID), zap.Stringer("lock", lock)) diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index d5f2fd7093523..b1fc5d69a9e5e 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -23,7 +23,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -39,7 +38,6 @@ var _ = Suite(&testCommitterSuite{}) func (s *testCommitterSuite) SetUpTest(c *C) { s.cluster = mocktikv.NewCluster() - PessimisticLockTTL = uint64(config.MinPessimisticTTL / time.Millisecond) mocktikv.BootstrapWithMultiRegions(s.cluster, []byte("a"), []byte("b"), []byte("c")) mvccStore, err := mocktikv.NewMVCCLevelDB("") c.Assert(err, IsNil) @@ -492,18 +490,3 @@ func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { c.Assert(err, IsNil) c.Assert(txn.lockKeys, HasLen, 2) } - -func (s *testCommitterSuite) TestPessimistOptimisticConflict(c *C) { - txnPes := s.begin(c) - txnPes.SetOption(kv.Pessimistic, true) - err := txnPes.LockKeys(context.Background(), txnPes.startTS, kv.Key("pes")) - c.Assert(err, IsNil) - c.Assert(txnPes.IsPessimistic(), IsTrue) - c.Assert(txnPes.lockKeys, HasLen, 1) - txnOpt := s.begin(c) - err = txnOpt.Set(kv.Key("pes"), []byte("v")) - c.Assert(err, IsNil) - err = txnOpt.Commit(context.Background()) - c.Assert(kv.ErrWriteConflict.Equal(err), IsTrue) - c.Assert(txnPes.Commit(context.Background()), IsNil) -} From 8dff08a88f5fb54e13632d396962d0e227c4be91 Mon Sep 17 00:00:00 2001 From: Tanner Date: Thu, 4 Jul 2019 21:46:27 +0800 Subject: [PATCH 15/21] =?UTF-8?q?ddl:=20disallow=20modifying=20the=20gener?= =?UTF-8?q?ated=20expression=20of=20stored=20or=E2=80=A6=20(#11068)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/db_test.go | 80 ++++++++++++++++++++++++++++++++++++++--- ddl/ddl_api.go | 16 +-------- ddl/generated_column.go | 59 ++++++++++++++++++++++++------ table/tables/tables.go | 2 +- 4 files changed, 126 insertions(+), 31 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index df9b12597f02c..55ba11897f0f5 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2121,13 +2121,17 @@ func (s *testDBSuite3) TestGeneratedColumnDDL(c *C) { } // Check alter table modify/change generated column. - s.tk.MustExec(`alter table test_gv_ddl modify column c bigint as (b+200) stored`) + modStoredColErrMsg := "[ddl:3106]'modifying a stored column' is not supported for generated columns." + _, err := s.tk.Exec(`alter table test_gv_ddl modify column c bigint as (b+200) stored`) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, modStoredColErrMsg) + result = s.tk.MustQuery(`DESC test_gv_ddl`) - result.Check(testkit.Rows(`a int(11) YES `, `b int(11) YES VIRTUAL GENERATED`, `c bigint(20) YES STORED GENERATED`)) + result.Check(testkit.Rows(`a int(11) YES `, `b int(11) YES VIRTUAL GENERATED`, `c int(11) YES STORED GENERATED`)) s.tk.MustExec(`alter table test_gv_ddl change column b b bigint as (a+100) virtual`) result = s.tk.MustQuery(`DESC test_gv_ddl`) - result.Check(testkit.Rows(`a int(11) YES `, `b bigint(20) YES VIRTUAL GENERATED`, `c bigint(20) YES STORED GENERATED`)) + result.Check(testkit.Rows(`a int(11) YES `, `b bigint(20) YES VIRTUAL GENERATED`, `c int(11) YES STORED GENERATED`)) s.tk.MustExec(`alter table test_gv_ddl change column c cnew bigint`) result = s.tk.MustQuery(`DESC test_gv_ddl`) @@ -2720,7 +2724,75 @@ func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { s.tk.MustExec("admin check table gcai_table") } -func (s *testDBSuite6) TestIssue9100(c *C) { +func (s *testDBSuite5) TestModifyGeneratedColumn(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test;") + tk.MustExec("use test") + modIdxColErrMsg := "[ddl:3106]'modifying an indexed column' is not supported for generated columns." + modStoredColErrMsg := "[ddl:3106]'modifying a stored column' is not supported for generated columns." + + // Modify column with single-col-index. + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") + tk.MustExec("insert into t1 set a=1;") + _, err := tk.Exec("alter table t1 modify column b int as (a+2);") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, modIdxColErrMsg) + tk.MustExec("drop index idx on t1;") + tk.MustExec("alter table t1 modify b int as (a+2);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) + + // Modify column with multi-col-index. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(a, b));") + tk.MustExec("insert into t1 set a=1;") + _, err = tk.Exec("alter table t1 modify column b int as (a+2);") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, modIdxColErrMsg) + tk.MustExec("drop index idx on t1;") + tk.MustExec("alter table t1 modify b int as (a+2);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) + + // Modify column with stored status to a different expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + _, err = tk.Exec("alter table t1 modify column b int as (a+2) stored;") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, modStoredColErrMsg) + + // Modify column with stored status to the same expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b bigint as (a+1) stored;") + tk.MustExec("alter table t1 modify column b bigint as (a + 1) stored;") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) + + // Modify column with index to the same expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b bigint as (a+1);") + tk.MustExec("alter table t1 modify column b bigint as (a + 1);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) + + // Modify column from non-generated to stored generated. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int);") + _, err = tk.Exec("alter table t1 modify column b bigint as (a+1) stored;") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, modStoredColErrMsg) + + // Modify column from stored generated to non-generated. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b int;") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) +} + +func (s *testDBSuite4) TestIssue9100(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db") tk.MustExec("create table employ (a int, b int) partition by range (b) (partition p0 values less than (1));") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 9900536c9ad7e..996a7f69831fe 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2551,7 +2551,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or } // As same with MySQL, we don't support modifying the stored status for generated columns. - if err = checkModifyGeneratedColumn(t.Cols(), col, newCol); err != nil { + if err = checkModifyGeneratedColumn(t, col, newCol, specNewColumn); err != nil { return nil, errors.Trace(err) } @@ -2604,20 +2604,6 @@ func (d *ddl) ModifyColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al return ErrWrongTableName.GenWithStackByArgs(specNewColumn.Name.Table.O) } - // If the modified column is generated, check whether it refers to any auto-increment columns. - for _, option := range specNewColumn.Options { - if option.Tp == ast.ColumnOptionGenerated { - _, t, err := d.getSchemaAndTableByIdent(ctx, ident) - if err != nil { - return errors.Trace(err) - } - _, dependColNames := findDependedColumnNames(specNewColumn) - if err := checkAutoIncrementRef(specNewColumn.Name.Name.L, dependColNames, t.Meta()); err != nil { - return errors.Trace(err) - } - } - } - originalColName := specNewColumn.Name.Name job, err := d.getModifiableColumnJob(ctx, ident, originalColName, spec) if err != nil { diff --git a/ddl/generated_column.go b/ddl/generated_column.go index e0caa6ae6b075..e9ee788ff033b 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -36,7 +36,7 @@ func verifyColumnGeneration(colName2Generation map[string]columnGenerationInDDL, if attr, ok := colName2Generation[depCol]; ok { if attr.generated && attribute.position <= attr.position { // A generated column definition can refer to other - // generated columns occurring earilier in the table. + // generated columns occurring earlier in the table. err := errGeneratedColumnNonPrior.GenWithStackByArgs() return errors.Trace(err) } @@ -109,19 +109,18 @@ func (c *generatedColumnChecker) Leave(inNode ast.Node) (node ast.Node, ok bool) // 1. the modification can't change stored status; // 2. if the new is generated, check its refer rules. // 3. check if the modified expr contains non-deterministic functions -func checkModifyGeneratedColumn(originCols []*table.Column, oldCol, newCol *table.Column) error { +// 4. check whether new column refers to any auto-increment columns. +// 5. check if the new column is indexed or stored +func checkModifyGeneratedColumn(tbl table.Table, oldCol, newCol *table.Column, newColDef *ast.ColumnDef) error { // rule 1. - var stored = [2]bool{false, false} - var cols = [2]*table.Column{oldCol, newCol} - for i, col := range cols { - if !col.IsGenerated() || col.GeneratedStored { - stored[i] = true - } - } - if stored[0] != stored[1] { + oldColIsStored := !oldCol.IsGenerated() || oldCol.GeneratedStored + newColIsStored := !newCol.IsGenerated() || newCol.GeneratedStored + if oldColIsStored != newColIsStored { return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("Changing the STORED status") } + // rule 2. + originCols := tbl.Cols() var colName2Generation = make(map[string]columnGenerationInDDL, len(originCols)) for i, column := range originCols { // We can compare the pointers simply. @@ -158,11 +157,21 @@ func checkModifyGeneratedColumn(originCols []*table.Column, oldCol, newCol *tabl } } - // rule 3 if newCol.IsGenerated() { + // rule 3. if err := checkIllegalFn4GeneratedColumn(newCol.Name.L, newCol.GeneratedExpr); err != nil { return errors.Trace(err) } + + // rule 4. + if err := checkGeneratedWithAutoInc(tbl.Meta(), newColDef); err != nil { + return errors.Trace(err) + } + + // rule 5. + if err := checkIndexOrStored(tbl, oldCol, newCol); err != nil { + return errors.Trace(err) + } } return nil } @@ -198,6 +207,34 @@ func checkIllegalFn4GeneratedColumn(colName string, expr ast.ExprNode) error { return nil } +// Check whether newColumnDef refers to any auto-increment columns. +func checkGeneratedWithAutoInc(tableInfo *model.TableInfo, newColumnDef *ast.ColumnDef) error { + _, dependColNames := findDependedColumnNames(newColumnDef) + if err := checkAutoIncrementRef(newColumnDef.Name.Name.L, dependColNames, tableInfo); err != nil { + return errors.Trace(err) + } + return nil +} + +func checkIndexOrStored(tbl table.Table, oldCol, newCol *table.Column) error { + if oldCol.GeneratedExprString == newCol.GeneratedExprString { + return nil + } + + if newCol.GeneratedStored { + return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying a stored column") + } + + for _, idx := range tbl.Indices() { + for _, col := range idx.Meta().Columns { + if col.Name.L == newCol.Name.L { + return errUnsupportedOnGeneratedColumn.GenWithStackByArgs("modifying an indexed column") + } + } + } + return nil +} + // checkAutoIncrementRef checks if an generated column depends on an auto-increment column and raises an error if so. // See https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. func checkAutoIncrementRef(name string, dependencies map[string]struct{}, tbInfo *model.TableInfo) error { diff --git a/table/tables/tables.go b/table/tables/tables.go index d7ed18fbe3f97..b8eb936aefe1e 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1047,7 +1047,7 @@ var ( recordPrefixSep = []byte("_r") ) -// FindIndexByColName implements table.Table FindIndexByColName interface. +// FindIndexByColName returns a public table index containing only one column named `name`. func FindIndexByColName(t table.Table, name string) table.Index { for _, idx := range t.Indices() { // only public index can be read. From 8493e45ca93e10653e8c3d09dd37fb49d4df8aa4 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 5 Jul 2019 17:29:16 +0800 Subject: [PATCH 16/21] ddl: Remove expired keys on PD (#10406) (#11014) --- ddl/ddl.go | 22 +++- ddl/ddl_worker.go | 4 + ddl/mock.go | 14 ++- ddl/{ => util}/syncer.go | 138 +++++++++++++++++++++- ddl/util/syncer_test.go | 249 +++++++++++++++++++++++++++++++++++++++ domain/info.go | 6 +- go.mod | 9 +- go.sum | 19 +-- metrics/ddl.go | 12 +- metrics/metrics.go | 13 +- 10 files changed, 450 insertions(+), 36 deletions(-) rename ddl/{ => util}/syncer.go (75%) create mode 100644 ddl/util/syncer_test.go diff --git a/ddl/ddl.go b/ddl/ddl.go index ce8a593ed0c51..8cc1b5d840769 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -268,7 +268,7 @@ type DDL interface { // RegisterEventCh registers event channel for ddl. RegisterEventCh(chan<- *util.Event) // SchemaSyncer gets the schema syncer. - SchemaSyncer() SchemaSyncer + SchemaSyncer() util.SchemaSyncer // OwnerManager gets the owner manager. OwnerManager() owner.Manager // GetID gets the ddl ID. @@ -297,7 +297,7 @@ type ddlCtx struct { uuid string store kv.Storage ownerManager owner.Manager - schemaSyncer SchemaSyncer + schemaSyncer util.SchemaSyncer ddlJobDoneCh chan struct{} ddlEventCh chan<- *util.Event lease time.Duration // lease is schema lease. @@ -364,7 +364,7 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, id := uuid.NewV4().String() ctx, cancelFunc := context.WithCancel(ctx) var manager owner.Manager - var syncer SchemaSyncer + var syncer util.SchemaSyncer if etcdCli == nil { // The etcdCli is nil if the store is localstore which is only used for testing. // So we use mockOwnerManager and MockSchemaSyncer. @@ -372,7 +372,7 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, syncer = NewMockSchemaSyncer() } else { manager = owner.NewOwnerManager(etcdCli, ddlPrompt, id, DDLOwnerKey, cancelFunc) - syncer = NewSchemaSyncer(etcdCli, id) + syncer = util.NewSchemaSyncer(etcdCli, id, manager) } ddlCtx := &ddlCtx{ @@ -455,6 +455,17 @@ func (d *ddl) start(ctx context.Context, ctxPool *pools.ResourcePool) { // checks owner firstly and try to find whether a job exists and run. asyncNotify(worker.ddlJobCh) } + + go tidbutil.WithRecovery( + func() { d.schemaSyncer.StartCleanWork() }, + func(r interface{}) { + if r != nil { + logutil.Logger(ddlLogCtx).Error("[ddl] DDL syncer clean worker meet panic", + zap.String("ID", d.uuid), zap.Reflect("r", r), zap.Stack("stack trace")) + metrics.PanicCounter.WithLabelValues(metrics.LabelDDLSyncer).Inc() + } + }) + metrics.DDLCounter.WithLabelValues(fmt.Sprintf("%s", metrics.StartCleanWork)).Inc() } } @@ -466,6 +477,7 @@ func (d *ddl) close() { startTime := time.Now() close(d.quitCh) d.ownerManager.Cancel() + d.schemaSyncer.CloseCleanWork() err := d.schemaSyncer.RemoveSelfVersionPath() if err != nil { logutil.Logger(ddlLogCtx).Error("[ddl] remove self version path failed", zap.Error(err)) @@ -528,7 +540,7 @@ func (d *ddl) genGlobalIDs(count int) ([]int64, error) { } // SchemaSyncer implements DDL.SchemaSyncer interface. -func (d *ddl) SchemaSyncer() SchemaSyncer { +func (d *ddl) SchemaSyncer() util.SchemaSyncer { return d.schemaSyncer } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 6ec289f3ebd82..1b188e95cec8e 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -42,6 +42,8 @@ var ( RunWorker = true // ddlWorkerID is used for generating the next DDL worker ID. ddlWorkerID = int32(0) + // WaitTimeWhenErrorOccured is waiting interval when processing DDL jobs encounter errors. + WaitTimeWhenErrorOccured = 1 * time.Second ) type workerType byte @@ -645,6 +647,8 @@ func (w *worker) waitSchemaChanged(ctx context.Context, d *ddlCtx, waitTime time if terror.ErrorEqual(err, context.DeadlineExceeded) { return } + d.schemaSyncer.NotifyCleanExpiredPaths() + // Wait until timeout. select { case <-ctx.Done(): return diff --git a/ddl/mock.go b/ddl/mock.go index 1911f8aeec704..c73cb6f81cb51 100644 --- a/ddl/mock.go +++ b/ddl/mock.go @@ -22,10 +22,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/sessionctx" ) -var _ SchemaSyncer = &MockSchemaSyncer{} +var _ util.SchemaSyncer = &MockSchemaSyncer{} const mockCheckVersInterval = 2 * time.Millisecond @@ -37,7 +38,7 @@ type MockSchemaSyncer struct { } // NewMockSchemaSyncer creates a new mock SchemaSyncer. -func NewMockSchemaSyncer() SchemaSyncer { +func NewMockSchemaSyncer() util.SchemaSyncer { return &MockSchemaSyncer{} } @@ -113,6 +114,15 @@ func (s *MockSchemaSyncer) OwnerCheckAllVersions(ctx context.Context, latestVer } } +// NotifyCleanExpiredPaths implements SchemaSyncer.NotifyCleanExpiredPaths interface. +func (s *MockSchemaSyncer) NotifyCleanExpiredPaths() bool { return true } + +// StartCleanWork implements SchemaSyncer.StartCleanWork interface. +func (s *MockSchemaSyncer) StartCleanWork() {} + +// CloseCleanWork implements SchemaSyncer.CloseCleanWork interface. +func (s *MockSchemaSyncer) CloseCleanWork() {} + type mockDelRange struct { } diff --git a/ddl/syncer.go b/ddl/util/syncer.go similarity index 75% rename from ddl/syncer.go rename to ddl/util/syncer.go index da75b83242440..61305e7fb27ab 100644 --- a/ddl/syncer.go +++ b/ddl/util/syncer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package ddl +package util import ( "context" @@ -25,7 +25,9 @@ import ( "github.com/coreos/etcd/clientv3" "github.com/coreos/etcd/clientv3/concurrency" + "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/pingcap/errors" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/util/logutil" @@ -48,6 +50,8 @@ const ( keyOpDefaultTimeout = 2 * time.Second keyOpRetryInterval = 30 * time.Millisecond checkVersInterval = 20 * time.Millisecond + + ddlPrompt = "ddl-syncer" ) var ( @@ -57,8 +61,8 @@ var ( // SyncerSessionTTL is the etcd session's TTL in seconds. // and it's an exported variable for testing. SyncerSessionTTL = 90 - // WaitTimeWhenErrorOccured is waiting interval when processing DDL jobs encounter errors. - WaitTimeWhenErrorOccured = 1 * time.Second + // ddlLogCtx uses for log. + ddlLogCtx = context.Background() ) // SchemaSyncer is used to synchronize schema version between the DDL worker leader and followers through etcd. @@ -86,6 +90,17 @@ type SchemaSyncer interface { // the latest schema version. If the result is false, wait for a while and check again util the processing time reach 2 * lease. // It returns until all servers' versions are equal to the latest version or the ctx is done. OwnerCheckAllVersions(ctx context.Context, latestVer int64) error + // NotifyCleanExpiredPaths informs to clean up expired paths. + // The returned value is used for testing. + NotifyCleanExpiredPaths() bool + // StartCleanWork starts to clean up tasks. + StartCleanWork() + // CloseCleanWork ends cleanup tasks. + CloseCleanWork() +} + +type ownerChecker interface { + IsOwner() bool } type schemaVersionSyncer struct { @@ -96,13 +111,21 @@ type schemaVersionSyncer struct { sync.RWMutex globalVerCh clientv3.WatchChan } + + // for clean worker + ownerChecker ownerChecker + notifyCleanExpiredPathsCh chan struct{} + quiteCh chan struct{} } // NewSchemaSyncer creates a new SchemaSyncer. -func NewSchemaSyncer(etcdCli *clientv3.Client, id string) SchemaSyncer { +func NewSchemaSyncer(etcdCli *clientv3.Client, id string, oc ownerChecker) SchemaSyncer { return &schemaVersionSyncer{ - etcdCli: etcdCli, - selfSchemaVerPath: fmt.Sprintf("%s/%s", DDLAllSchemaVersions, id), + etcdCli: etcdCli, + selfSchemaVerPath: fmt.Sprintf("%s/%s", DDLAllSchemaVersions, id), + ownerChecker: oc, + notifyCleanExpiredPathsCh: make(chan struct{}, 1), + quiteCh: make(chan struct{}), } } @@ -380,3 +403,106 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, latestV time.Sleep(checkVersInterval) } } + +const ( + opDefaultRetryCnt = 10 + failedGetTTLLimit = 20 + opDefaultTimeout = 3 * time.Second + opRetryInterval = 500 * time.Millisecond +) + +// NeededCleanTTL is exported for testing. +var NeededCleanTTL = int64(-60) + +func (s *schemaVersionSyncer) StartCleanWork() { + for { + select { + case <-s.notifyCleanExpiredPathsCh: + if !s.ownerChecker.IsOwner() { + continue + } + + for i := 0; i < opDefaultRetryCnt; i++ { + childCtx, cancelFunc := context.WithTimeout(context.Background(), opDefaultTimeout) + resp, err := s.etcdCli.Leases(childCtx) + cancelFunc() + if err != nil { + logutil.Logger(ddlLogCtx).Info("[ddl] syncer clean expired paths, failed to get leases.", zap.Error(err)) + continue + } + + if isFinished := s.doCleanExpirePaths(resp.Leases); isFinished { + break + } + time.Sleep(opRetryInterval) + } + case <-s.quiteCh: + return + } + } +} + +func (s *schemaVersionSyncer) CloseCleanWork() { + close(s.quiteCh) +} + +func (s *schemaVersionSyncer) NotifyCleanExpiredPaths() bool { + var isNotified bool + var err error + startTime := time.Now() + select { + case s.notifyCleanExpiredPathsCh <- struct{}{}: + isNotified = true + default: + err = errors.New("channel is full, failed to notify clean expired paths") + } + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerNotifyCleanExpirePaths, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + return isNotified +} + +func (s *schemaVersionSyncer) doCleanExpirePaths(leases []clientv3.LeaseStatus) bool { + failedGetIDs := 0 + failedRevokeIDs := 0 + startTime := time.Now() + + defer func() { + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerCleanExpirePaths, metrics.RetLabel(nil)).Observe(time.Since(startTime).Seconds()) + }() + // TODO: Now LeaseStatus only has lease ID. + for _, lease := range leases { + // The DDL owner key uses '%x', so here print it too. + leaseID := fmt.Sprintf("%x, %d", lease.ID, lease.ID) + childCtx, cancelFunc := context.WithTimeout(context.Background(), opDefaultTimeout) + ttlResp, err := s.etcdCli.TimeToLive(childCtx, lease.ID) + cancelFunc() + if err != nil { + logutil.Logger(ddlLogCtx).Info("[ddl] syncer clean expired paths, failed to get one TTL.", zap.String("leaseID", leaseID), zap.Error(err)) + failedGetIDs++ + continue + } + + if failedGetIDs > failedGetTTLLimit { + return false + } + if ttlResp.TTL >= NeededCleanTTL { + continue + } + + st := time.Now() + childCtx, cancelFunc = context.WithTimeout(context.Background(), opDefaultTimeout) + _, err = s.etcdCli.Revoke(childCtx, lease.ID) + cancelFunc() + if err != nil && terror.ErrorEqual(err, rpctypes.ErrLeaseNotFound) { + logutil.Logger(ddlLogCtx).Warn("[ddl] syncer clean expired paths, failed to revoke lease.", zap.String("leaseID", leaseID), + zap.Int64("TTL", ttlResp.TTL), zap.Error(err)) + failedRevokeIDs++ + } + logutil.Logger(ddlLogCtx).Warn("[ddl] syncer clean expired paths,", zap.String("leaseID", leaseID), zap.Int64("TTL", ttlResp.TTL)) + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerCleanOneExpirePath, metrics.RetLabel(err)).Observe(time.Since(st).Seconds()) + } + + if failedGetIDs == 0 && failedRevokeIDs == 0 { + return true + } + return false +} diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go new file mode 100644 index 0000000000000..9199ba2ac2857 --- /dev/null +++ b/ddl/util/syncer_test.go @@ -0,0 +1,249 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util_test + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + "github.com/coreos/etcd/clientv3" + "github.com/coreos/etcd/etcdserver" + "github.com/coreos/etcd/integration" + "github.com/coreos/etcd/mvcc/mvccpb" + "github.com/pingcap/errors" + "github.com/pingcap/parser/terror" + . "github.com/pingcap/tidb/ddl" + . "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/store/mockstore" + goctx "golang.org/x/net/context" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" +) + +const minInterval = 10 * time.Nanosecond // It's used to test timeout. + +func TestSyncerSimple(t *testing.T) { + testLease := 5 * time.Millisecond + origin := CheckVersFirstWaitTime + CheckVersFirstWaitTime = 0 + defer func() { + CheckVersFirstWaitTime = origin + }() + + store, err := mockstore.NewMockTikvStore() + if err != nil { + t.Fatal(err) + } + defer store.Close() + + clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer clus.Terminate(t) + cli := clus.RandClient() + ctx := goctx.Background() + d := NewDDL(ctx, cli, store, nil, nil, testLease, nil) + defer d.Stop() + + // for init function + if err = d.SchemaSyncer().Init(ctx); err != nil { + t.Fatalf("schema version syncer init failed %v", err) + } + resp, err := cli.Get(ctx, DDLAllSchemaVersions, clientv3.WithPrefix()) + if err != nil { + t.Fatalf("client get version failed %v", err) + } + key := DDLAllSchemaVersions + "/" + d.OwnerManager().ID() + checkRespKV(t, 1, key, InitialVersion, resp.Kvs...) + // for MustGetGlobalVersion function + globalVer, err := d.SchemaSyncer().MustGetGlobalVersion(ctx) + if err != nil { + t.Fatalf("client get global version failed %v", err) + } + if InitialVersion != fmt.Sprintf("%d", globalVer) { + t.Fatalf("client get global version %d isn't equal to init version %s", globalVer, InitialVersion) + } + childCtx, _ := goctx.WithTimeout(ctx, minInterval) + _, err = d.SchemaSyncer().MustGetGlobalVersion(childCtx) + if !isTimeoutError(err) { + t.Fatalf("client get global version result not match, err %v", err) + } + + d1 := NewDDL(ctx, cli, store, nil, nil, testLease, nil) + defer d1.Stop() + if err = d1.SchemaSyncer().Init(ctx); err != nil { + t.Fatalf("schema version syncer init failed %v", err) + } + + // for watchCh + wg := sync.WaitGroup{} + wg.Add(1) + currentVer := int64(123) + go func() { + defer wg.Done() + select { + case resp := <-d.SchemaSyncer().GlobalVersionCh(): + if len(resp.Events) < 1 { + t.Fatalf("get chan events count less than 1") + } + checkRespKV(t, 1, DDLGlobalSchemaVersion, fmt.Sprintf("%v", currentVer), resp.Events[0].Kv) + case <-time.After(100 * time.Millisecond): + t.Fatalf("get udpate version failed") + } + }() + + // for update latestSchemaVersion + err = d.SchemaSyncer().OwnerUpdateGlobalVersion(ctx, currentVer) + if err != nil { + t.Fatalf("update latest schema version failed %v", err) + } + + wg.Wait() + + // for CheckAllVersions + childCtx, cancel := goctx.WithTimeout(ctx, 20*time.Millisecond) + err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer) + if err == nil { + t.Fatalf("check result not match") + } + cancel() + + // for UpdateSelfVersion + childCtx, cancel = goctx.WithTimeout(ctx, 100*time.Millisecond) + err = d.SchemaSyncer().UpdateSelfVersion(childCtx, currentVer) + if err != nil { + t.Fatalf("update self version failed %v", errors.ErrorStack(err)) + } + cancel() + childCtx, cancel = goctx.WithTimeout(ctx, 100*time.Millisecond) + err = d1.SchemaSyncer().UpdateSelfVersion(childCtx, currentVer) + if err != nil { + t.Fatalf("update self version failed %v", errors.ErrorStack(err)) + } + cancel() + childCtx, _ = goctx.WithTimeout(ctx, minInterval) + err = d1.SchemaSyncer().UpdateSelfVersion(childCtx, currentVer) + if !isTimeoutError(err) { + t.Fatalf("update self version result not match, err %v", err) + } + + // for CheckAllVersions + childCtx, _ = goctx.WithTimeout(ctx, 100*time.Millisecond) + err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer-1) + if err != nil { + t.Fatalf("check all versions failed %v", err) + } + childCtx, _ = goctx.WithTimeout(ctx, 100*time.Millisecond) + err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer) + if err != nil { + t.Fatalf("check all versions failed %v", err) + } + childCtx, _ = goctx.WithTimeout(ctx, minInterval) + err = d.SchemaSyncer().OwnerCheckAllVersions(childCtx, currentVer) + if !isTimeoutError(err) { + t.Fatalf("check all versions result not match, err %v", err) + } + + // for StartCleanWork + go d.SchemaSyncer().StartCleanWork() + ttl := 10 + // Make sure NeededCleanTTL > ttl, then we definitely clean the ttl. + NeededCleanTTL = int64(11) + ttlKey := "session_ttl_key" + ttlVal := "session_ttl_val" + session, err := owner.NewSession(ctx, "", cli, owner.NewSessionDefaultRetryCnt, ttl) + if err != nil { + t.Fatalf("new session failed %v", err) + } + childCtx, cancel = context.WithTimeout(ctx, 100*time.Millisecond) + err = PutKVToEtcd(childCtx, cli, 5, ttlKey, ttlVal, clientv3.WithLease(session.Lease())) + if err != nil { + t.Fatalf("put kv to etcd failed %v", err) + } + cancel() + // Make sure the ttlKey is exist in etcd. + resp, err = cli.Get(ctx, ttlKey) + if err != nil { + t.Fatalf("client get version failed %v", err) + } + checkRespKV(t, 1, ttlKey, ttlVal, resp.Kvs...) + d.SchemaSyncer().NotifyCleanExpiredPaths() + // Make sure the clean worker is done. + notifiedCnt := 1 + for i := 0; i < 100; i++ { + isNotified := d.SchemaSyncer().NotifyCleanExpiredPaths() + if isNotified { + notifiedCnt++ + } + // notifyCleanExpiredPathsCh's length is 1, + // so when notifiedCnt is 3, we can make sure the clean worker is done at least once. + if notifiedCnt == 3 { + break + } + time.Sleep(20 * time.Millisecond) + } + if notifiedCnt != 3 { + t.Fatal("clean worker don't finish") + } + // Make sure the ttlKey is removed in etcd. + resp, err = cli.Get(ctx, ttlKey) + if err != nil { + t.Fatalf("client get version failed %v", err) + } + checkRespKV(t, 0, ttlKey, "", resp.Kvs...) + + // for RemoveSelfVersionPath + resp, err = cli.Get(goctx.Background(), key) + if err != nil { + t.Fatalf("get key %s failed %v", key, err) + } + currVer := fmt.Sprintf("%v", currentVer) + checkRespKV(t, 1, key, currVer, resp.Kvs...) + d.SchemaSyncer().RemoveSelfVersionPath() + resp, err = cli.Get(goctx.Background(), key) + if err != nil { + t.Fatalf("get key %s failed %v", key, err) + } + if len(resp.Kvs) != 0 { + t.Fatalf("remove key %s failed %v", key, err) + } +} + +func isTimeoutError(err error) bool { + if terror.ErrorEqual(err, goctx.DeadlineExceeded) || grpc.Code(errors.Cause(err)) == codes.DeadlineExceeded || + terror.ErrorEqual(err, etcdserver.ErrTimeout) { + return true + } + return false +} + +func checkRespKV(t *testing.T, kvCount int, key, val string, + kvs ...*mvccpb.KeyValue) { + if len(kvs) != kvCount { + t.Fatalf("resp key %s kvs %v length is != %d", key, kvs, kvCount) + } + if kvCount == 0 { + return + } + + kv := kvs[0] + if string(kv.Key) != key { + t.Fatalf("key resp %s, exported %s", kv.Key, key) + } + if val != val { + t.Fatalf("val resp %s, exported %s", kv.Value, val) + } +} diff --git a/domain/info.go b/domain/info.go index 49d614c295ab3..0831c267f8a7f 100644 --- a/domain/info.go +++ b/domain/info.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" @@ -129,7 +129,7 @@ func (is *InfoSyncer) storeServerInfo(ctx context.Context) error { return errors.Trace(err) } str := string(hack.String(infoBuf)) - err = ddl.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.serverInfoPath, str, clientv3.WithLease(is.session.Lease())) + err = util.PutKVToEtcd(ctx, is.etcdCli, keyOpDefaultRetryCnt, is.serverInfoPath, str, clientv3.WithLease(is.session.Lease())) return err } @@ -138,7 +138,7 @@ func (is *InfoSyncer) RemoveServerInfo() { if is.etcdCli == nil { return } - err := ddl.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) + err := util.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) if err != nil { logutil.Logger(context.Background()).Error("remove server info failed", zap.Error(err)) } diff --git a/go.mod b/go.mod index f277f82c30489..9a9cf7c3a3441 100644 --- a/go.mod +++ b/go.mod @@ -5,10 +5,9 @@ require ( github.com/StackExchange/wmi v0.0.0-20180725035823-b12b22c5341f // indirect github.com/beorn7/perks v1.0.0 // indirect github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d - github.com/boltdb/bolt v1.3.1 // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd // indirect - github.com/coreos/bbolt v1.3.0 // indirect - github.com/coreos/etcd v3.3.12+incompatible + github.com/coreos/bbolt v1.3.3 // indirect + github.com/coreos/etcd v3.3.13+incompatible github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 @@ -27,8 +26,11 @@ require ( github.com/gorilla/websocket v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 github.com/grpc-ecosystem/grpc-gateway v1.5.1 // indirect + github.com/json-iterator/go v1.1.6 // indirect github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.1 // indirect github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 // indirect github.com/myesui/uuid v1.0.0 // indirect github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 @@ -64,6 +66,7 @@ require ( github.com/uber/jaeger-client-go v2.15.0+incompatible github.com/uber/jaeger-lib v1.5.0 // indirect github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d // indirect + go.etcd.io/bbolt v1.3.3 // indirect go.uber.org/atomic v1.3.2 go.uber.org/zap v1.9.1 golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e diff --git a/go.sum b/go.sum index 2658354e7ee2a..75096a0a46b7f 100644 --- a/go.sum +++ b/go.sum @@ -10,8 +10,6 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= -github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= -github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -19,10 +17,10 @@ github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/coreos/bbolt v1.3.0 h1:HIgH5xUWXT914HCI671AxuTTqjj64UOFr7pHn48LUTI= -github.com/coreos/bbolt v1.3.0/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.12+incompatible h1:pAWNwdf7QiT1zfaWyqCtNZQWCLByQyA3JrSQyuYAqnQ= -github.com/coreos/etcd v3.3.12+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/bbolt v1.3.3 h1:n6AiVyVRKQFNb6mJlwESEvvLoDyiTzXX7ORAUlkeBdY= +github.com/coreos/bbolt v1.3.3/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= +github.com/coreos/etcd v3.3.13+incompatible h1:8F3hqu9fGYLBifCmRCJsicFqDx/D68Rt3q1JMazcgBQ= +github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -104,6 +102,8 @@ github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpO github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/json-iterator/go v1.1.6 h1:MrUvLMLTMxbqFJ9kzlvat/rYZqZnW3u4wkLzWTaFwKs= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= @@ -124,6 +124,10 @@ github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vq github.com/matttproud/golang_protobuf_extensions v1.0.0/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 h1:pmpDGKLw4n82EtrNiLqB+xSz/JQwFOaZuMALYUHwX5s= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= @@ -244,8 +248,9 @@ github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKn github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= -go.etcd.io/bbolt v1.3.2 h1:Z/90sZLPOeCy2PwprqkFa25PdkusRzaj9P8zm/KNyvk= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= +go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= +go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf h1:rmttwKPEgG/l4UscTDYtaJgeUsedKPKSyFfNQLI6q+I= go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf/go.mod h1:KSGwdbiFchh5KIC9My2+ZVl5/3ANcwohw50dpPwa2cw= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= diff --git a/metrics/ddl.go b/metrics/ddl.go index 8a6de6347cd38..92de5826c6d8c 100644 --- a/metrics/ddl.go +++ b/metrics/ddl.go @@ -65,10 +65,13 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1024s }, []string{LblResult}) - OwnerUpdateGlobalVersion = "update_global_version" - OwnerGetGlobalVersion = "get_global_version" - OwnerCheckAllVersions = "check_all_versions" - OwnerHandleSyncerHistogram = prometheus.NewHistogramVec( + OwnerUpdateGlobalVersion = "update_global_version" + OwnerGetGlobalVersion = "get_global_version" + OwnerCheckAllVersions = "check_all_versions" + OwnerNotifyCleanExpirePaths = "notify_clean_expire_paths" + OwnerCleanExpirePaths = "clean_expire_paths" + OwnerCleanOneExpirePath = "clean_an_expire_path" + OwnerHandleSyncerHistogram = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "ddl", @@ -93,6 +96,7 @@ var ( CreateDDLInstance = "create_ddl_instance" CreateDDL = "create_ddl" + StartCleanWork = "start_clean_work" DDLOwner = "owner" DDLCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ diff --git a/metrics/metrics.go b/metrics/metrics.go index 5711d411efc12..5cd1005be9a44 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -28,12 +28,13 @@ var ( // metrics labels. const ( - LabelSession = "session" - LabelDomain = "domain" - LabelDDLOwner = "ddl-owner" - LabelDDL = "ddl" - LabelGCWorker = "gcworker" - LabelAnalyze = "analyze" + LabelSession = "session" + LabelDomain = "domain" + LabelDDLOwner = "ddl-owner" + LabelDDL = "ddl" + LabelDDLSyncer = "ddl-syncer" + LabelGCWorker = "gcworker" + LabelAnalyze = "analyze" LabelBatchRecvLoop = "batch-recv-loop" LabelBatchSendLoop = "batch-send-loop" From c3346f2eb1614e19bf28ae043c3086e7aaac9edc Mon Sep 17 00:00:00 2001 From: bb7133 Date: Sat, 6 Jul 2019 12:30:48 +0800 Subject: [PATCH 17/21] =?UTF-8?q?ddl:=20allow=20more=20charset/collation?= =?UTF-8?q?=20modifications=20for=20database/t=E2=80=A6=20(#11086)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/db_integration_test.go | 31 +++++++++++++++++++------------ ddl/ddl_api.go | 6 ++++-- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 61a8e9f452fc5..317104492b459 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -668,17 +668,17 @@ func (s *testIntegrationSuite10) TestChangingTableCharset(c *C) { tk.MustExec("drop table t;") tk.MustExec("create table t(a varchar(10)) charset utf8") tk.MustExec("alter table t convert to charset utf8mb4;") - checkCharset := func() { + checkCharset := func(chs, coll string) { tbl := testGetTableByName(c, s.ctx, "test", "t") c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Charset, Equals, charset.CharsetUTF8MB4) - c.Assert(tbl.Meta().Collate, Equals, charset.CollationUTF8MB4) + c.Assert(tbl.Meta().Charset, Equals, chs) + c.Assert(tbl.Meta().Collate, Equals, coll) for _, col := range tbl.Meta().Columns { - c.Assert(col.Charset, Equals, charset.CharsetUTF8MB4) - c.Assert(col.Collate, Equals, charset.CollationUTF8MB4) + c.Assert(col.Charset, Equals, chs) + c.Assert(col.Collate, Equals, coll) } } - checkCharset() + checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) // Test when column charset can not convert to the target charset. tk.MustExec("drop table t;") @@ -687,11 +687,16 @@ func (s *testIntegrationSuite10) TestChangingTableCharset(c *C) { c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify charset from ascii to utf8mb4") + tk.MustExec("drop table t;") + tk.MustExec("create table t(a varchar(10) character set utf8) charset utf8") + tk.MustExec("alter table t convert to charset utf8 collate utf8_general_ci;") + checkCharset(charset.CharsetUTF8, "utf8_general_ci") + // Test when table charset is equal to target charset but column charset is not equal. tk.MustExec("drop table t;") tk.MustExec("create table t(a varchar(10) character set utf8) charset utf8mb4") - tk.MustExec("alter table t convert to charset utf8mb4;") - checkCharset() + tk.MustExec("alter table t convert to charset utf8mb4 collate utf8mb4_general_ci;") + checkCharset(charset.CharsetUTF8MB4, "utf8mb4_general_ci") // Mock table info with charset is "". Old TiDB maybe create table with charset is "". db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test")) @@ -724,7 +729,7 @@ func (s *testIntegrationSuite10) TestChangingTableCharset(c *C) { c.Assert(tbl.Meta().Collate, Equals, "") // Test when table charset is "", this for compatibility. tk.MustExec("alter table t convert to charset utf8mb4;") - checkCharset() + checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) // Test when column charset is "". tbl = testGetTableByName(c, s.ctx, "test", "t") @@ -739,7 +744,7 @@ func (s *testIntegrationSuite10) TestChangingTableCharset(c *C) { c.Assert(tbl.Meta().Columns[0].Charset, Equals, "") c.Assert(tbl.Meta().Columns[0].Collate, Equals, "") tk.MustExec("alter table t convert to charset utf8mb4;") - checkCharset() + checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) tk.MustExec("drop table t") tk.MustExec("create table t (a blob) character set utf8;") @@ -1749,10 +1754,12 @@ func (s *testIntegrationSuite11) TestChangingDBCharset(c *C) { for _, fc := range failedCases { c.Assert(tk.ExecToErr(fc.stmt).Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt)) } + tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8' COLLATE = 'utf8_unicode_ci'") + verifyDBCharsetAndCollate("alterdb2", "utf8", "utf8_unicode_ci") tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8mb4'") verifyDBCharsetAndCollate("alterdb2", "utf8mb4", "utf8mb4_bin") - err := tk.ExecToErr("ALTER SCHEMA CHARACTER SET = 'utf8mb4' COLLATE = 'utf8mb4_general_ci'") - c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify collate from utf8mb4_bin to utf8mb4_general_ci") + tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8mb4' COLLATE = 'utf8mb4_general_ci'") + verifyDBCharsetAndCollate("alterdb2", "utf8mb4", "utf8mb4_general_ci") } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 996a7f69831fe..8356d59269ae2 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2262,8 +2262,10 @@ func modifiableCharsetAndCollation(toCharset, toCollate, origCharset, origCollat if !charset.ValidCharsetAndCollation(toCharset, toCollate) { return ErrUnknownCharacterSet.GenWithStack("Unknown character set: '%s', collation: '%s'", toCharset, toCollate) } - if toCharset == charset.CharsetUTF8MB4 && origCharset == charset.CharsetUTF8 { - // TiDB only allow utf8 to be changed to utf8mb4. + if (origCharset == charset.CharsetUTF8 && toCharset == charset.CharsetUTF8MB4) || + (origCharset == charset.CharsetUTF8 && toCharset == charset.CharsetUTF8) || + (origCharset == charset.CharsetUTF8MB4 && toCharset == charset.CharsetUTF8MB4) { + // TiDB only allow utf8 to be changed to utf8mb4, or changing the collation when the charset is utf8/utf8mb4. return nil } From 731e426facfc72e9f46212ee0fcf5f4c7f877c9e Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Sat, 6 Jul 2019 12:40:42 +0800 Subject: [PATCH 18/21] executor: handle unsigned primary key for fast analyze (#11074) (#11099) --- executor/analyze.go | 3 +++ executor/analyze_test.go | 7 +++++++ 2 files changed, 10 insertions(+) diff --git a/executor/analyze.go b/executor/analyze.go index 54bbef8bfe8d4..7158425d5a430 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -813,6 +813,9 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam } v = types.NewIntDatum(key) } + if mysql.HasUnsignedFlag(e.pkInfo.Flag) { + v.SetUint64(uint64(v.GetInt64())) + } if e.collectors[0].Samples[samplePos] == nil { e.collectors[0].Samples[samplePos] = &statistics.SampleItem{} } diff --git a/executor/analyze_test.go b/executor/analyze_test.go index c2cf8d9fef34b..3216e294f5838 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -290,6 +290,13 @@ func (s *testSuite1) TestFastAnalyze(c *C) { tk.MustQuery("explain select a, b from t1 where a = 1 and b = 2").Check(testkit.Rows( "IndexReader_6 2.00 root index:IndexScan_5", "└─IndexScan_5 2.00 cop table:t1, index:a, b, range:[1 2,1 2], keep order:false")) + + tk.MustExec("create table t2 (a bigint unsigned, primary key(a))") + tk.MustExec("insert into t2 values (0), (18446744073709551615)") + tk.MustExec("analyze table t2") + tk.MustQuery("show stats_buckets where table_name = 't2'").Check(testkit.Rows( + "test t2 a 0 0 1 1 0 0", + "test t2 a 0 1 2 1 18446744073709551615 18446744073709551615")) } func (s *testSuite1) TestAnalyzeIncremental(c *C) { From e0c69f0d719a941eac4905056d6661749b974362 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Sat, 6 Jul 2019 12:44:12 +0800 Subject: [PATCH 19/21] executor: fix invalid key error of fast analyze (#11072) (#11098) --- executor/analyze.go | 4 +++- executor/analyze_test.go | 46 +++++++++++++++++----------------------- 2 files changed, 23 insertions(+), 27 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 7158425d5a430..b0568abefd789 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -965,7 +965,9 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e if *err != nil { return } - kvMap[string(iter.Key())] = iter.Value() + if iter.Valid() { + kvMap[string(iter.Key())] = iter.Value() + } } *err = e.handleBatchSeekResponse(kvMap) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 3216e294f5838..ab4ef000fde30 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -229,7 +229,7 @@ func (s *testSuite1) TestFastAnalyze(c *C) { dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) tk := testkit.NewTestKit(c, store) - executor.MaxSampleSize = 1000 + executor.MaxSampleSize = 6 executor.RandSeed = 123 tk.MustExec("use test") @@ -241,12 +241,12 @@ func (s *testSuite1) TestFastAnalyze(c *C) { c.Assert(err, IsNil) tid := tblInfo.Meta().ID - // construct 5 regions split by {600, 1200, 1800, 2400} - splitKeys := generateTableSplitKeyForInt(tid, []int{600, 1200, 1800, 2400}) + // construct 6 regions split by {10, 20, 30, 40, 50} + splitKeys := generateTableSplitKeyForInt(tid, []int{10, 20, 30, 40, 50}) manipulateCluster(cluster, splitKeys) - for i := 0; i < 3000; i++ { - tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "char")`, i, i)) + for i := 0; i < 20; i++ { + tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "char")`, i*3, i*3)) } tk.MustExec("analyze table t with 5 buckets") @@ -255,27 +255,21 @@ func (s *testSuite1) TestFastAnalyze(c *C) { c.Assert(err, IsNil) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) - c.Assert(tbl.String(), Equals, "Table:41 Count:3000\n"+ - "column:1 ndv:3000 totColSize:0\n"+ - "num: 603 lower_bound: 0 upper_bound: 658 repeats: 1\n"+ - "num: 603 lower_bound: 663 upper_bound: 1248 repeats: 1\n"+ - "num: 603 lower_bound: 1250 upper_bound: 1823 repeats: 1\n"+ - "num: 603 lower_bound: 1830 upper_bound: 2379 repeats: 1\n"+ - "num: 588 lower_bound: 2380 upper_bound: 2998 repeats: 1\n"+ - "column:2 ndv:3000 totColSize:0\n"+ - "num: 603 lower_bound: 0 upper_bound: 658 repeats: 1\n"+ - "num: 603 lower_bound: 663 upper_bound: 1248 repeats: 1\n"+ - "num: 603 lower_bound: 1250 upper_bound: 1823 repeats: 1\n"+ - "num: 603 lower_bound: 1830 upper_bound: 2379 repeats: 1\n"+ - "num: 588 lower_bound: 2380 upper_bound: 2998 repeats: 1\n"+ - "column:3 ndv:1 totColSize:12000\n"+ - "num: 3000 lower_bound: char upper_bound: char repeats: 3000\n"+ - "index:1 ndv:3000\n"+ - "num: 603 lower_bound: 0 upper_bound: 658 repeats: 1\n"+ - "num: 603 lower_bound: 663 upper_bound: 1248 repeats: 1\n"+ - "num: 603 lower_bound: 1250 upper_bound: 1823 repeats: 1\n"+ - "num: 603 lower_bound: 1830 upper_bound: 2379 repeats: 1\n"+ - "num: 588 lower_bound: 2380 upper_bound: 2998 repeats: 1") + c.Assert(tbl.String(), Equals, "Table:41 Count:20\n"+ + "column:1 ndv:20 totColSize:0\n"+ + "num: 6 lower_bound: 3 upper_bound: 15 repeats: 1\n"+ + "num: 7 lower_bound: 18 upper_bound: 33 repeats: 1\n"+ + "num: 7 lower_bound: 39 upper_bound: 57 repeats: 1\n"+ + "column:2 ndv:20 totColSize:0\n"+ + "num: 6 lower_bound: 3 upper_bound: 15 repeats: 1\n"+ + "num: 7 lower_bound: 18 upper_bound: 33 repeats: 1\n"+ + "num: 7 lower_bound: 39 upper_bound: 57 repeats: 1\n"+ + "column:3 ndv:1 totColSize:72\n"+ + "num: 20 lower_bound: char upper_bound: char repeats: 18\n"+ + "index:1 ndv:20\n"+ + "num: 6 lower_bound: 3 upper_bound: 15 repeats: 1\n"+ + "num: 7 lower_bound: 18 upper_bound: 33 repeats: 1\n"+ + "num: 7 lower_bound: 39 upper_bound: 57 repeats: 1") // Test CM Sketch built from fast analyze. tk.MustExec("create table t1(a int, b int, index idx(a, b))") From b43bdb74a3ca3f416db75565e19cf26625a10ea3 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Sat, 6 Jul 2019 12:50:07 +0800 Subject: [PATCH 20/21] =?UTF-8?q?executor,=20infoschema:=20fix=20display?= =?UTF-8?q?=20of=20default=20CURRENT=5FTIMESTAMP=E2=80=A6=20(#11088)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- executor/show.go | 5 ++++- executor/show_test.go | 29 +++++++++++++++++++++++++++++ infoschema/tables_test.go | 36 ++++++++++++++++++++++++++++++++++++ table/column.go | 8 ++++++++ 4 files changed, 77 insertions(+), 1 deletion(-) diff --git a/executor/show.go b/executor/show.go index 5618f04b20c75..37fad01831aaf 100644 --- a/executor/show.go +++ b/executor/show.go @@ -402,7 +402,7 @@ func (e *ShowExec) fetchShowColumns() error { // SHOW COLUMNS result expects string value defaultValStr := fmt.Sprintf("%v", desc.DefaultValue) // If column is timestamp, and default value is not current_timestamp, should convert the default value to the current session time zone. - if col.Tp == mysql.TypeTimestamp && defaultValStr != types.ZeroDatetimeStr && strings.ToUpper(defaultValStr) != strings.ToUpper(ast.CurrentTimestamp) { + if col.Tp == mysql.TypeTimestamp && defaultValStr != types.ZeroDatetimeStr && !strings.HasPrefix(strings.ToUpper(defaultValStr), strings.ToUpper(ast.CurrentTimestamp)) { timeValue, err := table.GetColDefaultValue(e.ctx, col.ToInfo()) if err != nil { return errors.Trace(err) @@ -692,6 +692,9 @@ func (e *ShowExec) fetchShowCreateTable() error { } case "CURRENT_TIMESTAMP": buf.WriteString(" DEFAULT CURRENT_TIMESTAMP") + if col.Decimal > 0 { + buf.WriteString(fmt.Sprintf("(%d)", col.Decimal)) + } default: defaultValStr := fmt.Sprintf("%v", defaultValue) // If column is timestamp, and default value is not current_timestamp, should convert the default value to the current session time zone. diff --git a/executor/show_test.go b/executor/show_test.go index c34d357c5f825..a1c8afc6548f6 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -186,13 +186,18 @@ func (s *testSuite2) TestShow2(c *C) { c_nchar national char(1) charset ascii collate ascii_bin, c_binary binary, c_varchar varchar(1) charset ascii collate ascii_bin, + c_varchar_default varchar(20) charset ascii collate ascii_bin default 'cUrrent_tImestamp', c_nvarchar national varchar(1) charset ascii collate ascii_bin, c_varbinary varbinary(1), c_year year, c_date date, c_time time, c_datetime datetime, + c_datetime_default datetime default current_timestamp, + c_datetime_default_2 datetime(2) default current_timestamp(2), c_timestamp timestamp, + c_timestamp_default timestamp default current_timestamp, + c_timestamp_default_3 timestamp(3) default current_timestamp(3), c_blob blob, c_tinyblob tinyblob, c_mediumblob mediumblob, @@ -216,13 +221,18 @@ func (s *testSuite2) TestShow2(c *C) { "[c_nchar char(1) ascii_bin YES select,insert,update,references ]\n" + "[c_binary binary(1) YES select,insert,update,references ]\n" + "[c_varchar varchar(1) ascii_bin YES select,insert,update,references ]\n" + + "[c_varchar_default varchar(20) ascii_bin YES cUrrent_tImestamp select,insert,update,references ]\n" + "[c_nvarchar varchar(1) ascii_bin YES select,insert,update,references ]\n" + "[c_varbinary varbinary(1) YES select,insert,update,references ]\n" + "[c_year year(4) YES select,insert,update,references ]\n" + "[c_date date YES select,insert,update,references ]\n" + "[c_time time YES select,insert,update,references ]\n" + "[c_datetime datetime YES select,insert,update,references ]\n" + + "[c_datetime_default datetime YES CURRENT_TIMESTAMP select,insert,update,references ]\n" + + "[c_datetime_default_2 datetime(2) YES CURRENT_TIMESTAMP(2) select,insert,update,references ]\n" + "[c_timestamp timestamp YES select,insert,update,references ]\n" + + "[c_timestamp_default timestamp YES CURRENT_TIMESTAMP select,insert,update,references ]\n" + + "[c_timestamp_default_3 timestamp(3) YES CURRENT_TIMESTAMP(3) select,insert,update,references ]\n" + "[c_blob blob YES select,insert,update,references ]\n" + "[c_tinyblob tinyblob YES select,insert,update,references ]\n" + "[c_mediumblob mediumblob YES select,insert,update,references ]\n" + @@ -451,6 +461,25 @@ func (s *testSuite2) TestShowCreateTable(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) + tk.MustExec("drop table if exists t") + tk.MustExec("create table `t` (\n" + + "`a` timestamp not null default current_timestamp,\n" + + "`b` timestamp(3) default current_timestamp(3),\n" + + "`c` datetime default current_timestamp,\n" + + "`d` datetime(4) default current_timestamp(4),\n" + + "`e` varchar(20) default 'cUrrent_tImestamp')") + tk.MustQuery("show create table `t`").Check(testutil.RowsWithSep("|", + ""+ + "t CREATE TABLE `t` (\n"+ + " `a` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,\n"+ + " `b` timestamp(3) DEFAULT CURRENT_TIMESTAMP(3),\n"+ + " `c` datetime DEFAULT CURRENT_TIMESTAMP,\n"+ + " `d` datetime(4) DEFAULT CURRENT_TIMESTAMP(4),\n"+ + " `e` varchar(20) DEFAULT 'cUrrent_tImestamp'\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + tk.MustExec("drop table t") + tk.MustExec("create table t (a int, b int) shard_row_id_bits = 4 pre_split_regions=3;") tk.MustQuery("show create table `t`").Check(testutil.RowsWithSep("|", ""+ diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 057ec4082ed25..ce102e67bc1a0 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -241,6 +241,42 @@ func (s *testTableSuite) TestCharacterSetCollations(c *C) { tk.MustExec("DROP DATABASE charset_collate_test") } +func (s *testTableSuite) TestCurrentTimestampAsDefault(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("DROP DATABASE IF EXISTS default_time_test") + tk.MustExec("CREATE DATABASE default_time_test; USE default_time_test") + + tk.MustExec(`CREATE TABLE default_time_table( + c_datetime datetime, + c_datetime_default datetime default current_timestamp, + c_datetime_default_2 datetime(2) default current_timestamp(2), + c_timestamp timestamp, + c_timestamp_default timestamp default current_timestamp, + c_timestamp_default_3 timestamp(3) default current_timestamp(3), + c_varchar_default varchar(20) default "current_timestamp", + c_varchar_default_3 varchar(20) default "current_timestamp(3)", + c_varchar_default_with_case varchar(20) default "cUrrent_tImestamp" + );`) + + tk.MustQuery(`SELECT column_name, column_default + FROM information_schema.COLUMNS + WHERE table_schema = "default_time_test" AND table_name = "default_time_table" + ORDER BY column_name`, + ).Check(testkit.Rows( + "c_datetime ", + "c_datetime_default CURRENT_TIMESTAMP", + "c_datetime_default_2 CURRENT_TIMESTAMP(2)", + "c_timestamp ", + "c_timestamp_default CURRENT_TIMESTAMP", + "c_timestamp_default_3 CURRENT_TIMESTAMP(3)", + "c_varchar_default current_timestamp", + "c_varchar_default_3 current_timestamp(3)", + "c_varchar_default_with_case cUrrent_tImestamp", + )) + tk.MustExec("DROP DATABASE default_time_test") +} + type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo } diff --git a/table/column.go b/table/column.go index ce9fb65454673..08f4884ad5d9c 100644 --- a/table/column.go +++ b/table/column.go @@ -19,6 +19,7 @@ package table import ( "context" + "fmt" "strings" "time" "unicode/utf8" @@ -256,6 +257,13 @@ func NewColDesc(col *Column) *ColDesc { var defaultValue interface{} if !mysql.HasNoDefaultValueFlag(col.Flag) { defaultValue = col.GetDefaultValue() + if defaultValStr, ok := defaultValue.(string); ok { + if (col.Tp == mysql.TypeTimestamp || col.Tp == mysql.TypeDatetime) && + strings.ToUpper(defaultValStr) == strings.ToUpper(ast.CurrentTimestamp) && + col.Decimal > 0 { + defaultValue = fmt.Sprintf("%s(%d)", defaultValStr, col.Decimal) + } + } } extra := "" From e2426c3dc1496509ccbe4f21e5f8f74d79ace9d1 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Sat, 6 Jul 2019 12:56:00 +0800 Subject: [PATCH 21/21] =?UTF-8?q?executor:=20let=20flush=20privileges=20do?= =?UTF-8?q?=20nothing=20when=20`skip-grant-tab=E2=80=A6=20(#11027)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- executor/executor_test.go | 1 + executor/simple.go | 7 +++++++ executor/simple_test.go | 31 +++++++++++++++++++++++++++++-- 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 2b5aaa145f229..227f67602de65 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -94,6 +94,7 @@ var _ = Suite(&testBypassSuite{}) var _ = Suite(&testUpdateSuite{}) var _ = Suite(&testOOMSuite{}) var _ = Suite(&testPointGetSuite{}) +var _ = Suite(&testFlushSuite{}) type testSuite struct { cluster *mocktikv.Cluster diff --git a/executor/simple.go b/executor/simple.go index 9aef2dc611495..be97225d5629a 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -812,6 +812,13 @@ func (e *SimpleExec) executeFlush(s *ast.FlushStmt) error { return errors.New("FLUSH TABLES WITH READ LOCK is not supported. Please use @@tidb_snapshot") } case ast.FlushPrivileges: + // If skip-grant-table is configured, do not flush privileges. + // Because LoadPrivilegeLoop does not run and the privilege Handle is nil, + // Call dom.PrivilegeHandle().Update would panic. + if config.GetGlobalConfig().Security.SkipGrantTable { + return nil + } + dom := domain.GetDomain(e.ctx) sysSessionPool := dom.SysSessionPool() ctx, err := sysSessionPool.Get() diff --git a/executor/simple_test.go b/executor/simple_test.go index cbbf4a76d921d..185a64cc6833f 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -16,17 +16,19 @@ package executor_test import ( "context" - "github.com/pingcap/tidb/planner/core" - . "github.com/pingcap/check" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -386,6 +388,31 @@ func (s *testSuite3) TestFlushPrivileges(c *C) { // After flush. _, err = se.Execute(ctx, `SELECT Password FROM mysql.User WHERE User="testflush" and Host="localhost"`) c.Check(err, IsNil) + +} + +type testFlushSuite struct{} + +func (s *testFlushSuite) TestFlushPrivilegesPanic(c *C) { + // Run in a separate suite because this test need to set SkipGrantTable config. + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + mvccStore := mocktikv.MustNewMVCCStore() + store, err := mockstore.NewMockTikvStore( + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), + ) + c.Assert(err, IsNil) + defer store.Close() + + config.GetGlobalConfig().Security.SkipGrantTable = true + dom, err := session.BootstrapSession(store) + c.Assert(err, IsNil) + defer dom.Close() + + tk := testkit.NewTestKit(c, store) + tk.MustExec("FLUSH PRIVILEGES") + config.GetGlobalConfig().Security.SkipGrantTable = false } func (s *testSuite3) TestDropStats(c *C) {