Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into dev/fix…
Browse files Browse the repository at this point in the history
…-join-generator
  • Loading branch information
zz-jason committed Aug 6, 2018
2 parents 6b74d45 + 44e6c3c commit 0ee08ad
Show file tree
Hide file tree
Showing 21 changed files with 231 additions and 110 deletions.
21 changes: 11 additions & 10 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -112,13 +112,14 @@ MemTableScan_4 10000.00 root
explain select c2 = (select c2 from t2 where t1.c1 = t2.c1 order by c1 limit 1) from t1;
id count task operator info
Projection_12 10000.00 root eq(test.t1.c2, test.t2.c2)
└─Apply_14 10000.00 root left outer join, inner:Limit_22
└─Apply_14 10000.00 root left outer join, inner:Limit_21
├─TableReader_16 10000.00 root data:TableScan_15
│ └─TableScan_15 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo
└─Limit_22 1.00 root offset:0, count:1
└─IndexLookUp_47 0.00 root
├─IndexScan_45 0.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo
└─TableScan_46 0.00 cop table:t2, keep order:false, stats:pseudo
└─Limit_21 1.00 root offset:0, count:1
└─IndexLookUp_43 1.00 root
├─Limit_42 1.00 cop offset:0, count:1
│ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo
└─TableScan_41 1.00 cop table:t2, keep order:false
explain select * from t1 order by c1 desc limit 1;
id count task operator info
Limit_10 1.00 root offset:0, count:1
Expand Down Expand Up @@ -286,8 +287,8 @@ Projection_11 10000.00 root 9_aux_0
│ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
└─StreamAgg_20 1.00 root funcs:count(1)
└─IndexJoin_32 10000.00 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a
├─IndexReader_36 10.00 root index:IndexScan_35
│ └─IndexScan_35 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo
├─IndexReader_36 10000.00 root index:IndexScan_35
│ └─IndexScan_35 10000.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo
└─TableReader_31 10.00 root data:TableScan_30
└─TableScan_30 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo
explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t;
Expand All @@ -298,9 +299,9 @@ Projection_11 10000.00 root 9_aux_0
│ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo
└─StreamAgg_20 1.00 root funcs:count(1)
└─IndexJoin_33 10000.00 root inner join, inner:TableReader_32, outer key:s.c, inner key:t1.a
├─IndexLookUp_38 10.00 root
│ ├─IndexScan_36 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo
│ └─TableScan_37 10.00 cop table:t, keep order:false, stats:pseudo
├─IndexLookUp_38 10000.00 root
│ ├─IndexScan_36 10000.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo
│ └─TableScan_37 10000.00 cop table:t, keep order:false, stats:pseudo
└─TableReader_32 10.00 root data:TableScan_31
└─TableScan_31 10.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo
drop table if exists t;
Expand Down
11 changes: 6 additions & 5 deletions cmd/explaintest/r/explain_easy_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -101,13 +101,14 @@ MemTableScan_4 10000.00 root
explain select c2 = (select c2 from t2 where t1.c1 = t2.c1 order by c1 limit 1) from t1;
id count task operator info
Projection_12 1999.00 root eq(test.t1.c2, test.t2.c2)
└─Apply_14 1999.00 root left outer join, inner:Limit_22
└─Apply_14 1999.00 root left outer join, inner:Limit_21
├─TableReader_16 1999.00 root data:TableScan_15
│ └─TableScan_15 1999.00 cop table:t1, range:[-inf,+inf], keep order:false
└─Limit_22 1.00 root offset:0, count:1
└─IndexLookUp_47 0.00 root
├─IndexScan_45 0.00 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableScan_46 0.00 cop table:t2, keep order:false
└─Limit_21 1.00 root offset:0, count:1
└─IndexLookUp_43 1.00 root
├─Limit_42 1.00 cop offset:0, count:1
│ └─IndexScan_40 1.25 cop table:t2, index:c1, range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableScan_41 1.00 cop table:t2, keep order:false
explain select * from t1 order by c1 desc limit 1;
id count task operator info
Limit_10 1.00 root offset:0, count:1
Expand Down
54 changes: 54 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2144,6 +2144,60 @@ func (s *testDBSuite) TestYearTypeCreateTable(c *C) {
c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsFalse)
}

func (s *testDBSuite) TestCheckColumnDefaultValue(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test;")
s.tk.MustExec("drop table if exists text_default_text;")
s.testErrorCode(c, "create table text_default_text(c1 text not null default '');", tmysql.ErrBlobCantHaveDefault)
s.testErrorCode(c, "create table text_default_text(c1 text not null default 'scds');", tmysql.ErrBlobCantHaveDefault)

s.tk.MustExec("drop table if exists text_default_json;")
s.testErrorCode(c, "create table text_default_json(c1 json not null default '');", tmysql.ErrBlobCantHaveDefault)
s.testErrorCode(c, "create table text_default_json(c1 json not null default 'dfew555');", tmysql.ErrBlobCantHaveDefault)

s.tk.MustExec("drop table if exists text_default_blob;")
s.testErrorCode(c, "create table text_default_blob(c1 blob not null default '');", tmysql.ErrBlobCantHaveDefault)
s.testErrorCode(c, "create table text_default_blob(c1 blob not null default 'scds54');", tmysql.ErrBlobCantHaveDefault)

s.tk.MustExec("set sql_mode='';")
s.tk.MustExec("drop table if exists text_default_text;")
s.tk.MustExec("create table text_default_text(c1 text not null default '');")
s.tk.MustQuery(`show create table text_default_text`).Check(testutil.RowsWithSep("|",
"text_default_text CREATE TABLE `text_default_text` (\n"+
" `c1` text NOT NULL\n"+
") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin",
))
ctx := s.tk.Se.(sessionctx.Context)
is := domain.GetDomain(ctx).InfoSchema()
tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_text"))
c.Assert(err, IsNil)
c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, "")

s.tk.MustExec("drop table if exists text_default_blob;")
s.tk.MustExec("create table text_default_blob(c1 blob not null default '');")
s.tk.MustQuery(`show create table text_default_blob`).Check(testutil.RowsWithSep("|",
"text_default_blob CREATE TABLE `text_default_blob` (\n"+
" `c1` blob NOT NULL\n"+
") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin",
))
is = domain.GetDomain(ctx).InfoSchema()
tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_blob"))
c.Assert(err, IsNil)
c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, "")

s.tk.MustExec("drop table if exists text_default_json;")
s.tk.MustExec("create table text_default_json(c1 json not null default '');")
s.tk.MustQuery(`show create table text_default_json`).Check(testutil.RowsWithSep("|",
"text_default_json CREATE TABLE `text_default_json` (\n"+
" `c1` json NOT NULL DEFAULT 'null'\n"+
") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin",
))
is = domain.GetDomain(ctx).InfoSchema()
tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_json"))
c.Assert(err, IsNil)
c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, `null`)
}

func (s *testDBSuite) TestCharacterSetInColumns(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("drop database if exists varchar_test;")
Expand Down
34 changes: 24 additions & 10 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,16 +251,32 @@ func buildColumnAndConstraint(ctx sessionctx.Context, offset int,
return col, cts, nil
}

// checkColumnCantHaveDefaultValue checks the column can have value as default or not.
// Now, TEXT/BLOB/JSON can't have not null value as default.
func checkColumnCantHaveDefaultValue(col *table.Column, value interface{}) (err error) {
// checkColumnDefaultValue checks the default value of the column.
// In non-strict SQL mode, if the default value of the column is an empty string, the default value can be ignored.
// In strict SQL mode, TEXT/BLOB/JSON can't have not null default values.
func checkColumnDefaultValue(ctx sessionctx.Context, col *table.Column, value interface{}) (bool, interface{}, error) {
hasDefaultValue := true
if value != nil && (col.Tp == mysql.TypeJSON ||
col.Tp == mysql.TypeTinyBlob || col.Tp == mysql.TypeMediumBlob ||
col.Tp == mysql.TypeLongBlob || col.Tp == mysql.TypeBlob) {
// TEXT/BLOB/JSON can't have not null default values.
return errBlobCantHaveDefault.GenByArgs(col.Name.O)
// In non-strict SQL mode.
if !ctx.GetSessionVars().SQLMode.HasStrictMode() && value == "" {
if col.Tp == mysql.TypeBlob || col.Tp == mysql.TypeLongBlob {
// The TEXT/BLOB default value can be ignored.
hasDefaultValue = false
}
// In non-strict SQL mode, if the column type is json and the default value is null, it is initialized to an empty array.
if col.Tp == mysql.TypeJSON {
value = `null`
}
sc := ctx.GetSessionVars().StmtCtx
sc.AppendWarning(errBlobCantHaveDefault.GenByArgs(col.Name.O))
return hasDefaultValue, value, nil
}
// In strict SQL mode or default value is not an empty string.
return hasDefaultValue, value, errBlobCantHaveDefault.GenByArgs(col.Name.O)
}
return nil
return hasDefaultValue, value, nil
}

// isExplicitTimeStamp is used to check if explicit_defaults_for_timestamp is on or off.
Expand Down Expand Up @@ -326,11 +342,10 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o
if err != nil {
return nil, nil, ErrColumnBadNull.Gen("invalid default value - %s", err)
}
if err = checkColumnCantHaveDefaultValue(col, value); err != nil {
if hasDefaultValue, value, err = checkColumnDefaultValue(ctx, col, value); err != nil {
return nil, nil, errors.Trace(err)
}
col.DefaultValue = value
hasDefaultValue = true
removeOnUpdateNowFlag(col)
case ast.ColumnOptionOnUpdate:
// TODO: Support other time functions.
Expand Down Expand Up @@ -1462,11 +1477,10 @@ func setDefaultAndComment(ctx sessionctx.Context, col *table.Column, options []*
if err != nil {
return ErrColumnBadNull.Gen("invalid default value - %s", err)
}
if err = checkColumnCantHaveDefaultValue(col, value); err != nil {
if hasDefaultValue, value, err = checkColumnDefaultValue(ctx, col, value); err != nil {
return errors.Trace(err)
}
col.DefaultValue = value
hasDefaultValue = true
case ast.ColumnOptionComment:
err := setColumnComment(ctx, col, opt)
if err != nil {
Expand Down
1 change: 0 additions & 1 deletion ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,6 @@ func (d *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl tab

builder.Request.NotFillCache = true
builder.Request.Priority = kv.PriorityLow
builder.Request.IsolationLevel = kv.SI

kvReq, err := builder.Build()
sctx := newContext(d.store)
Expand Down
15 changes: 4 additions & 11 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"math"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
Expand Down Expand Up @@ -125,15 +124,9 @@ func (builder *RequestBuilder) SetKeepOrder(order bool) *RequestBuilder {
return builder
}

func (builder *RequestBuilder) getIsolationLevel(sv *variable.SessionVars) kv.IsoLevel {
var isoLevel string
if sv.TxnIsolationLevelOneShot.State == 2 {
isoLevel = sv.TxnIsolationLevelOneShot.Value
}
if isoLevel == "" {
isoLevel, _ = sv.GetSystemVar(variable.TxnIsolation)
}
if isoLevel == ast.ReadCommitted {
func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel {
switch builder.Tp {
case kv.ReqTypeAnalyze:
return kv.RC
}
return kv.SI
Expand All @@ -155,7 +148,7 @@ func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int {
// "Concurrency", "IsolationLevel", "NotFillCache".
func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *RequestBuilder {
builder.Request.Concurrency = sv.DistSQLScanConcurrency
builder.Request.IsolationLevel = builder.getIsolationLevel(sv)
builder.Request.IsolationLevel = builder.getIsolationLevel()
builder.Request.NotFillCache = sv.StmtCtx.NotFillCache
builder.Request.Priority = builder.getKVPriority(sv)
return builder
Expand Down
2 changes: 1 addition & 1 deletion distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -507,7 +507,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) {
KeepOrder: true,
Desc: false,
Concurrency: 15,
IsolationLevel: 0,
IsolationLevel: kv.RC,
Priority: 1,
NotFillCache: true,
SyncLog: false,
Expand Down
10 changes: 10 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -459,4 +459,14 @@ func (s *testSuite) TestAdminCheckTable(c *C) {
tk.MustExec(fmt.Sprintf("insert into test values (%d, %d);", i, i))
}
tk.MustExec(`admin check table test;`)

// Test prefix index.
tk.MustExec(`drop table if exists t`)
tk.MustExec(`CREATE TABLE t (
ID CHAR(32) NOT NULL,
name CHAR(32) NOT NULL,
value CHAR(255),
INDEX indexIDname (ID(8),name(8)));`)
tk.MustExec(`INSERT INTO t VALUES ('keyword','urlprefix','text/ /text');`)
tk.MustExec(`admin check table t;`)
}
20 changes: 14 additions & 6 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/juju/errors"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
Expand All @@ -43,11 +42,12 @@ type AnalyzeExec struct {
tasks []*analyzeTask
}

var maxBucketSize = int64(256)

const (
maxSampleSize = 10000
maxRegionSampleSize = 1000
maxSketchSize = 10000
maxBucketSize = 256
defaultCMSketchDepth = 5
defaultCMSketchWidth = 2048
)
Expand Down Expand Up @@ -176,7 +176,6 @@ func (e *AnalyzeIndexExec) open() error {
SetKeepOrder(true).
Build()
kvReq.Concurrency = e.concurrency
kvReq.IsolationLevel = kv.RC
ctx := context.TODO()
e.result, err = distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars)
if err != nil {
Expand Down Expand Up @@ -212,7 +211,7 @@ func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statis
if err != nil {
return nil, nil, errors.Trace(err)
}
hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, statistics.HistogramFromProto(resp.Hist), maxBucketSize)
hist, err = statistics.MergeHistograms(e.ctx.GetSessionVars().StmtCtx, hist, statistics.HistogramFromProto(resp.Hist), int(maxBucketSize))
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down Expand Up @@ -291,7 +290,6 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe
SetAnalyzeRequest(e.analyzePB).
SetKeepOrder(e.keepOrder).
Build()
kvReq.IsolationLevel = kv.RC
kvReq.Concurrency = e.concurrency
if err != nil {
return nil, errors.Trace(err)
Expand Down Expand Up @@ -341,7 +339,7 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []
}
sc := e.ctx.GetSessionVars().StmtCtx
if e.pkInfo != nil {
pkHist, err = statistics.MergeHistograms(sc, pkHist, statistics.HistogramFromProto(resp.PkHist), maxBucketSize)
pkHist, err = statistics.MergeHistograms(sc, pkHist, statistics.HistogramFromProto(resp.PkHist), int(maxBucketSize))
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down Expand Up @@ -376,3 +374,13 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []
}
return hists, cms, nil
}

// SetMaxBucketSizeForTest sets the `maxBucketSize`.
func SetMaxBucketSizeForTest(size int64) {
maxBucketSize = size
}

// GetMaxBucketSizeForTest gets the `maxBucketSize`.
func GetMaxBucketSizeForTest() int64 {
return maxBucketSize
}
8 changes: 0 additions & 8 deletions executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/terror"
Expand Down Expand Up @@ -181,13 +180,6 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e
log.Infof("con:%d %s=%s", sessionVars.ConnectionID, name, valStr)
}

if name == variable.TxnIsolation {
isoLevel, _ := sessionVars.GetSystemVar(variable.TxnIsolation)
if isoLevel == ast.ReadCommitted {
e.ctx.Txn().SetOption(kv.IsolationLevel, kv.RC)
}
}

return nil
}

Expand Down
35 changes: 35 additions & 0 deletions plan/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/session"
Expand Down Expand Up @@ -620,6 +621,40 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) {
))
}

func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) {
defer testleak.AfterTest(c)()
store, dom, err := newStoreWithBootstrap()
c.Assert(err, IsNil)
tk := testkit.NewTestKit(c, store)
defer func() {
dom.Close()
store.Close()
}()
tk.MustExec("use test")
tk.MustExec("create table t(a int, b int, c int, index ab(a,b), index ac(a,c))")
tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)")
for i := 0; i < 10; i++ {
tk.MustExec("insert into t values (5,5,5), (10,10,10)")
}
origin := executor.GetMaxBucketSizeForTest()
defer func() { executor.SetMaxBucketSizeForTest(origin) }()
executor.SetMaxBucketSizeForTest(2)
tk.MustExec("analyze table t")
// Force using the histogram to estimate.
tk.MustExec("update mysql.stats_histograms set stats_ver = 0")
dom.StatsHandle().Clear()
dom.StatsHandle().Update(dom.InfoSchema())
// Using the histogram (a, b) to estimate `a = 5` will get 1.22, while using the CM Sketch to estimate
// the `a = 5 and c = 5` will get 10, it is not consistent.
tk.MustQuery("explain select * from t use index(ab) where a = 5 and c = 5").
Check(testkit.Rows(
"IndexLookUp_8 10.00 root ",
"├─IndexScan_5 12.50 cop table:t, index:a, b, range:[5,5], keep order:false",
"└─Selection_7 10.00 cop eq(test.t.c, 5)",
" └─TableScan_6 12.50 cop table:t, keep order:false",
))
}

func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) {
store, err := mockstore.NewMockTikvStore()
if err != nil {
Expand Down
Loading

0 comments on commit 0ee08ad

Please sign in to comment.