Skip to content

Commit

Permalink
Merge branch 'release-3.0' of github.com:pingcap/tidb into idle-recyc…
Browse files Browse the repository at this point in the history
…le-cherry
  • Loading branch information
tiancaiamao committed Jun 4, 2019
2 parents a30148c + 9cd7ea0 commit d95ddd4
Show file tree
Hide file tree
Showing 43 changed files with 662 additions and 132 deletions.
52 changes: 52 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,58 @@
# 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-rc.2] 2019-05-28
### SQL Optimizer
* Support Index Join in more scenarios
[#10540](https://github.com/pingcap/tidb/pull/10540)
* Support exporting historical statistics [#10291](https://github.com/pingcap/tidb/pull/10291)
* Support the incremental `Analyze` operation on monotonically increasing index columns
[#10355](https://github.com/pingcap/tidb/pull/10355)
* Neglect the NULL value in the `Order By` clause [#10488](https://github.com/pingcap/tidb/pull/10488)
* Fix the wrong schema information calculation of the `UnionAll` logical operator when simplifying the column information [#10384](https://github.com/pingcap/tidb/pull/10384)
* Avoid modifying the original expression when pushing down the `Not` operator [#10363](https://github.com/pingcap/tidb/pull/10363/files)
* Support the `dump`/`load` correlation of histograms [#10573](https://github.com/pingcap/tidb/pull/10573)
### Execution Engine
* Handle virtual columns with a unique index properly when fetching duplicate rows in `batchChecker` [#10370](https://github.com/pingcap/tidb/pull/10370)
* Fix the scanning range calculation issue for the `CHAR` column [#10124](https://github.com/pingcap/tidb/pull/10124)
* Fix the issue of `PointGet` incorrectly processing negative numbers [#10113](https://github.com/pingcap/tidb/pull/10113)
* Merge `Window` functions with the same name to improve execution efficiency [#9866](https://github.com/pingcap/tidb/pull/9866)
* Allow the `RANGE` frame in a `Window` function to contain no `OrderBy` clause [#10496](https://github.com/pingcap/tidb/pull/10496)

### Server
Fix the issue that TiDB continuously creates a new connection to TiKV when a fault occurs in TiKV [#10301](https://github.com/pingcap/tidb/pull/10301)
Make `tidb_disable_txn_auto_retry` affect all retryable errors instead of only write conflict errors [#10339](https://github.com/pingcap/tidb/pull/10339)
Allow DDL statements without parameters to be executed using `prepare`/`execute` [#10144](https://github.com/pingcap/tidb/pull/10144)
Add the `tidb_back_off_weight` variable to control the backoff time [#10266](https://github.com/pingcap/tidb/pull/10266)
Prohibit TiDB retrying non-automatically committed transactions in default conditions by setting the default value of `tidb_disable_txn_auto_retry` to `on` [#10266](https://github.com/pingcap/tidb/pull/10266)
Fix the database privilege judgment of `role` in `RBAC` [#10261](https://github.com/pingcap/tidb/pull/10261)
Support the pessimistic transaction model (experimental) [#10297](https://github.com/pingcap/tidb/pull/10297)
Reduce the wait time for handling lock conflicts in some cases [#10006](https://github.com/pingcap/tidb/pull/10006)
Make the Region cache able to visit follower nodes when a fault occurs in the leader node [#10256](https://github.com/pingcap/tidb/pull/10256)
Add the `tidb_low_resolution_tso` variable to control the number of TSOs obtained in batches and reduce the times of transactions obtaining TSO to adapt for scenarios where data consistency is not so strictly required [#10428](https://github.com/pingcap/tidb/pull/10428)

### DDL
Fix the uppercase issue of the charset name in the storage of the old version of TiDB
[#10272](https://github.com/pingcap/tidb/pull/10272)
Support `preSplit` of table partition, which pre-allocates table Regions when creating a table to avoid write hotspots after the table is created
[#10221](https://github.com/pingcap/tidb/pull/10221)
Fix the issue that TiDB incorrectly updates the version information in PD in some cases [#10324](https://github.com/pingcap/tidb/pull/10324)
Support modifying the charset and collation using the `ALTER DATABASE` statement
[#10393](https://github.com/pingcap/tidb/pull/10393)
Support splitting Regions based on the index and range of the specified table to relieve hotspot issues
[#10203](https://github.com/pingcap/tidb/pull/10203)
Prohibit modifying the precision of the decimal column using the `alter table` statement
[#10433](https://github.com/pingcap/tidb/pull/10433)
Fix the restriction for expressions and functions in hash partition
[#10273](https://github.com/pingcap/tidb/pull/10273)
Fix the issue that adding indexes in a table that contains partitions will in some cases cause TiDB panic
[#10475](https://github.com/pingcap/tidb/pull/10475)
Validate table information before executing the DDL to avoid invalid table schemas
[#10464](https://github.com/pingcap/tidb/pull/10464)
Enable hash partition by default; and enable range columns partition when there is only one column in the partition definition
[#9936](https://github.com/pingcap/tidb/pull/9936)


## [3.0.0-rc.1] 2019-05-10

### SQL Optimizer
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,7 +479,7 @@ func (d *ddl) close() {
d.delRangeMgr.clear()
}

logutil.Logger(ddlLogCtx).Info("[ddl] closing DDL", zap.String("ID", d.uuid), zap.Duration("takeTime", time.Since(startTime)))
logutil.Logger(ddlLogCtx).Info("[ddl] DDL closed", zap.String("ID", d.uuid), zap.Duration("take time", time.Since(startTime)))
}

// GetLease implements DDL.GetLease interface.
Expand Down
8 changes: 6 additions & 2 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,9 +103,10 @@ func (w *worker) String() string {
}

func (w *worker) close() {
startTime := time.Now()
close(w.quitCh)
w.wg.Wait()
logutil.Logger(w.logCtx).Info("[ddl] close DDL worker")
logutil.Logger(w.logCtx).Info("[ddl] DDL worker closed", zap.Duration("take time", time.Since(startTime)))
}

// start is used for async online schema changing, it will try to become the owner firstly,
Expand Down Expand Up @@ -637,7 +638,10 @@ func (w *worker) waitSchemaChanged(ctx context.Context, d *ddlCtx, waitTime time
return
}
}
logutil.Logger(w.logCtx).Info("[ddl] wait latest schema version changed", zap.Int64("ver", latestSchemaVersion), zap.Duration("takeTime", time.Since(timeStart)), zap.String("job", job.String()))
logutil.Logger(w.logCtx).Info("[ddl] wait latest schema version changed",
zap.Int64("ver", latestSchemaVersion),
zap.Duration("take time", time.Since(timeStart)),
zap.String("job", job.String()))
}

// waitSchemaSynced handles the following situation:
Expand Down
3 changes: 2 additions & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,6 +535,7 @@ func (do *Domain) mustReload() (exitLoop bool) {

// Close closes the Domain and release its resource.
func (do *Domain) Close() {
startTime := time.Now()
if do.ddl != nil {
terror.Log(do.ddl.Stop())
}
Expand All @@ -548,7 +549,7 @@ func (do *Domain) Close() {
do.sysSessionPool.Close()
do.slowQuery.Close()
do.wg.Wait()
logutil.Logger(context.Background()).Info("domain closed")
logutil.Logger(context.Background()).Info("domain closed", zap.Duration("take time", time.Since(startTime)))
}

type ddlCallback struct {
Expand Down
10 changes: 10 additions & 0 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -789,6 +789,7 @@ func (e *StreamAggExec) Open(ctx context.Context) error {
// Close implements the Executor Close interface.
func (e *StreamAggExec) Close() error {
e.childResult = nil
e.groupChecker.reset()
return e.baseExecutor.Close()
}

Expand Down Expand Up @@ -954,3 +955,12 @@ func (e *groupChecker) meetNewGroup(row chunk.Row) (bool, error) {
}
return !firstGroup, nil
}

func (e *groupChecker) reset() {
if e.curGroupKey != nil {
e.curGroupKey = e.curGroupKey[:0]
}
if e.tmpGroupKey != nil {
e.tmpGroupKey = e.tmpGroupKey[:0]
}
}
11 changes: 11 additions & 0 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -710,3 +710,14 @@ func (s *testSuite1) TestIssue10098(c *C) {
tk.MustExec("insert into t values('1', '222'), ('12', '22')")
tk.MustQuery("select group_concat(distinct a, b) from t").Check(testkit.Rows("1222,1222"))
}

func (s *testSuite1) TestIssue10608(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec(`drop table if exists t, s;`)
tk.MustExec("create table t(a int)")
tk.MustExec("create table s(a int, b int)")
tk.MustExec("insert into s values(100292, 508931), (120002, 508932)")
tk.MustExec("insert into t values(508931), (508932)")
tk.MustQuery("select (select group_concat(concat(123,'-')) from t where t.a = s.b group by t.a) as t from s;").Check(testkit.Rows("123-", "123-"))

}
10 changes: 8 additions & 2 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,7 +702,6 @@ func (e *AnalyzeFastExec) getNextSampleKey(bo *tikv.Backoffer, startKey kv.Key)
func (e *AnalyzeFastExec) buildSampTask() (needRebuild bool, err error) {
// Do get regions row count.
bo := tikv.NewBackoffer(context.Background(), 500)
e.rowCount = 0
needRebuildForRoutine := make([]bool, e.concurrency)
errs := make([]error, e.concurrency)
sampTasksForRoutine := make([][]*AnalyzeFastTask, e.concurrency)
Expand Down Expand Up @@ -734,6 +733,13 @@ func (e *AnalyzeFastExec) buildSampTask() (needRebuild bool, err error) {
if err != nil {
return false, err
}
e.rowCount = 0
for _, task := range e.sampTasks {
cnt := task.EndOffset - task.BeginOffset
task.BeginOffset = e.rowCount
task.EndOffset = e.rowCount + cnt
e.rowCount += cnt
}
for {
// Search for the region which contains the targetKey.
loc, err := e.cache.LocateKey(bo, targetKey)
Expand Down Expand Up @@ -949,7 +955,7 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e
keys = append(keys, tablecodec.EncodeRowKeyWithHandle(tableID, randKey))
}

var kvMap map[string][]byte
kvMap := make(map[string][]byte, len(keys))
for _, key := range keys {
var iter kv.Iterator
iter, *err = snapshot.Iter(key, endKey)
Expand Down
68 changes: 23 additions & 45 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,9 +237,6 @@ func (s *testSuite1) TestFastAnalyze(c *C) {
tk.MustExec("create table t(a int primary key, b int, index index_b(b))")
tk.MustExec("set @@session.tidb_enable_fast_analyze=1")
tk.MustExec("set @@session.tidb_build_stats_concurrency=1")
for i := 0; i < 3000; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
}
tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tid := tblInfo.Meta().ID
Expand All @@ -248,56 +245,35 @@ func (s *testSuite1) TestFastAnalyze(c *C) {
splitKeys := generateTableSplitKeyForInt(tid, []int{600, 1200, 1800, 2400})
manipulateCluster(cluster, splitKeys)

for i := 0; i < 3000; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
}
tk.MustExec("analyze table t with 5 buckets")

is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
tbl := dom.StatsHandle().GetTableStats(tableInfo)
sTbl := fmt.Sprintln(tbl)
matched := false
if sTbl == "Table:39 Count:3000\n"+
c.Assert(tbl.String(), Equals, "Table:39 Count:3000\n"+
"column:1 ndv:3000 totColSize:0\n"+
"num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\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: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\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"+
"index:1 ndv:3000\n"+
"num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" ||
sTbl == "Table:39 Count:3000\n"+
"column:2 ndv:3000 totColSize:0\n"+
"num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+
"column:1 ndv:3000 totColSize:0\n"+
"num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+
"index:1 ndv:3000\n"+
"num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+
"num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+
"num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+
"num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+
"num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" {
matched = true
}
c.Assert(matched, Equals, true)
"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")
}

func (s *testSuite1) TestAnalyzeIncremental(c *C) {
Expand Down Expand Up @@ -415,7 +391,7 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(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 int, index index_b(b))")
tk.MustExec("create table t(a int primary key)")
tk.MustExec("set @@session.tidb_enable_fast_analyze=1")
tk.MustExec("set @@session.tidb_build_stats_concurrency=1")
tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
Expand All @@ -425,12 +401,14 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) {
splitKeys := generateTableSplitKeyForInt(tid, []int{6, 12, 18, 24, 30})
regionIDs := manipulateCluster(s.cluster, splitKeys)
for i := 0; i < 30; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
tk.MustExec(fmt.Sprintf("insert into t values (%d)", i))
}
s.cli.setFailRegion(regionIDs[4])
tk.MustExec("analyze table t")
// 4 regions will be sampled, and it will retry the last failed region.
c.Assert(s.cli.mu.count, Equals, int64(5))
row := tk.MustQuery(`show stats_meta where db_name = "test" and table_name = "t"`).Rows()[0]
c.Assert(row[5], Equals, "30")
}

func (s *testSuite1) TestFailedAnalyzeRequest(c *C) {
Expand Down
7 changes: 6 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -550,7 +550,12 @@ func (b *executorBuilder) buildShow(v *plannercore.Show) Executor {
is: b.is,
}
if e.Tp == ast.ShowGrants && e.User == nil {
e.User = e.ctx.GetSessionVars().User
// The input is a "show grants" statement, fulfill the user and roles field.
// Note: "show grants" result are different from "show grants for current_user",
// The former determine privileges with roles, while the later doesn't.
vars := e.ctx.GetSessionVars()
e.User = vars.User
e.Roles = vars.ActiveRoles
}
if e.Tp == ast.ShowMasterStatus {
// show master status need start ts.
Expand Down
2 changes: 2 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1380,6 +1380,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.NotFillCache = !opts.SQLCache
}
sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode()
case *ast.ExplainStmt:
sc.InExplainStmt = true
case *ast.ShowStmt:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
Expand Down
23 changes: 20 additions & 3 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/planner"
plannercore "github.com/pingcap/tidb/planner/core"
Expand Down Expand Up @@ -226,9 +227,9 @@ func (s *testSuite) TestAdmin(c *C) {
result.Check(testkit.Rows())
result = tk.MustQuery(`admin show ddl job queries 1, 2, 3, 4`)
result.Check(testkit.Rows())
historyJob, err := admin.GetHistoryDDLJobs(txn, admin.DefNumHistoryJobs)
result = tk.MustQuery(fmt.Sprintf("admin show ddl job queries %d", historyJob[0].ID))
result.Check(testkit.Rows(historyJob[0].Query))
historyJobs, err = admin.GetHistoryDDLJobs(txn, admin.DefNumHistoryJobs)
result = tk.MustQuery(fmt.Sprintf("admin show ddl job queries %d", historyJobs[0].ID))
result.Check(testkit.Rows(historyJobs[0].Query))
c.Assert(err, IsNil)

// check table test
Expand Down Expand Up @@ -282,6 +283,22 @@ func (s *testSuite) TestAdmin(c *C) {
tk.MustExec("ALTER TABLE t1 ADD COLUMN c4 bit(10) default 127;")
tk.MustExec("ALTER TABLE t1 ADD INDEX idx3 (c4);")
tk.MustExec("admin check table t1;")

// Test for reverse scan get history ddl jobs when ddl history jobs queue has multiple regions.
txn, err = s.store.Begin()
c.Assert(err, IsNil)
historyJobs, err = admin.GetHistoryDDLJobs(txn, 20)
c.Assert(err, IsNil)

// Split region for history ddl job queues.
m := meta.NewMeta(txn)
startKey := meta.DDLJobHistoryKey(m, 0)
endKey := meta.DDLJobHistoryKey(m, historyJobs[0].ID)
s.cluster.SplitKeys(s.mvccStore, startKey, endKey, int(historyJobs[0].ID/5))

historyJobs2, err := admin.GetHistoryDDLJobs(txn, 20)
c.Assert(err, IsNil)
c.Assert(historyJobs, DeepEquals, historyJobs2)
}

func (s *testSuite) fillData(tk *testkit.TestKit, table string) {
Expand Down
Loading

0 comments on commit d95ddd4

Please sign in to comment.