Skip to content

Commit

Permalink
Merge branch 'master' into sql-binding
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao committed Jun 15, 2021
2 parents cc1701a + 1ad33d8 commit 87caedb
Show file tree
Hide file tree
Showing 374 changed files with 12,113 additions and 4,803 deletions.
2 changes: 1 addition & 1 deletion bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,14 +38,14 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
utilparser "github.com/pingcap/tidb/util/parser"
"github.com/pingcap/tidb/util/stmtsummary"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
dto "github.com/prometheus/client_model/go"
"github.com/tikv/client-go/v2/mockstore/cluster"
)

func TestT(t *testing.T) {
Expand Down
4 changes: 2 additions & 2 deletions cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store"
"github.com/pingcap/tidb/store/driver"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/util/logutil"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/zap"
)

Expand All @@ -54,7 +54,7 @@ var (
func main() {
flag.Parse()
flag.PrintDefaults()
err := logutil.InitZapLogger(logutil.NewLogConfig(*logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(*logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
terror.MustNil(err)
err = store.Register("tikv", driver.TiKVDriver{})
terror.MustNil(err)
Expand Down
2 changes: 1 addition & 1 deletion cmd/benchfilesort/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,7 @@ func driveRunCmd() {
}

func init() {
err := logutil.InitZapLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
terror.MustNil(err)
cwd, err1 := os.Getwd()
terror.MustNil(err1)
Expand Down
4 changes: 2 additions & 2 deletions cmd/benchraw/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/config"
"github.com/tikv/client-go/v2/config"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/zap"
)

Expand Down
2 changes: 1 addition & 1 deletion cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ func openDBWithRetry(driverName, dataSourceName string) (mdb *sql.DB, err error)
func main() {
flag.Parse()

err := logutil.InitZapLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
if err != nil {
panic("init logger fail, " + err.Error())
}
Expand Down
23 changes: 23 additions & 0 deletions cmd/explaintest/r/cte.result
Original file line number Diff line number Diff line change
Expand Up @@ -147,3 +147,26 @@ create table t1(c1 bigint unsigned);
insert into t1 values(0);
with recursive cte1 as (select c1 - 1 c1 from t1 union all select c1 - 1 c1 from cte1 where c1 != 0) select * from cte1 dt1, cte1 dt2;
Error 1690: BIGINT UNSIGNED value is out of range in '(test.t1.c1 - 1)'
drop table if exists t;
create table t(a int, b int, key (b));
desc with cte as (select * from t) select * from cte;
id estRows task access object operator info
CTEFullScan_9 1.00 root CTE:cte data:CTE_0
CTE_0 1.00 root Non-Recursive CTE
└─TableReader_8(Seed Part) 10000.00 root data:TableFullScan_7
└─TableFullScan_7 10000.00 cop[tikv] table:t keep order:false, stats:pseudo
create SESSION binding for with cte as (select * from t) select * from cte using with cte as (select * from t use index(b)) select * from cte;
desc with cte as (select * from t) select * from cte;
id estRows task access object operator info
CTEFullScan_10 1.00 root CTE:cte data:CTE_0
CTE_0 1.00 root Non-Recursive CTE
└─IndexLookUp_9(Seed Part) 10000.00 root
├─IndexFullScan_7(Build) 10000.00 cop[tikv] table:t, index:b(b) keep order:false, stats:pseudo
└─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:t keep order:false, stats:pseudo
desc with cte as (select * from t use index()) select * from cte;
id estRows task access object operator info
CTEFullScan_10 1.00 root CTE:cte data:CTE_0
CTE_0 1.00 root Non-Recursive CTE
└─IndexLookUp_9(Seed Part) 10000.00 root
├─IndexFullScan_7(Build) 10000.00 cop[tikv] table:t, index:b(b) keep order:false, stats:pseudo
└─TableRowIDScan_8(Probe) 10000.00 cop[tikv] table:t keep order:false, stats:pseudo
11 changes: 6 additions & 5 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -124,11 +124,12 @@ Projection 10000.00 root eq(test.t1.c2, test.t2.c2)->Column#11
└─Apply 10000.00 root CARTESIAN left outer join
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo
└─Projection(Probe) 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp 1.00 root limit embedded(offset:0, count:1)
├─Limit(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan 1.00 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─Limit(Probe) 1.00 root offset:0, count:1
└─Projection 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp 1.00 root
├─Limit(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan 1.00 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
explain format = 'brief' select * from t1 order by c1 desc limit 1;
id estRows task access object operator info
Limit 1.00 root offset:0, count:1
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 @@ -106,11 +106,12 @@ Projection 1999.00 root eq(test.t1.c2, test.t2.c2)->Column#11
└─Apply 1999.00 root CARTESIAN left outer join
├─TableReader(Build) 1999.00 root data:TableFullScan
│ └─TableFullScan 1999.00 cop[tikv] table:t1 keep order:false
└─Projection(Probe) 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp 1.00 root limit embedded(offset:0, count:1)
├─Limit(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─Limit(Probe) 1.00 root offset:0, count:1
└─Projection 1.00 root test.t2.c1, test.t2.c2
└─IndexLookUp 1.00 root
├─Limit(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
explain format = 'brief' select * from t1 order by c1 desc limit 1;
id estRows task access object operator info
Limit 1.00 root offset:0, count:1
Expand Down
6 changes: 3 additions & 3 deletions cmd/explaintest/r/subquery.result
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,9 @@ Projection 5.00 root Column#22
├─TableReader(Build) 5.00 root data:TableFullScan
│ └─TableFullScan 5.00 cop[tikv] table:t keep order:false
└─StreamAgg(Probe) 1.00 root funcs:count(1)->Column#21
└─IndexJoin 0.50 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)
├─IndexReader(Build) 1.00 root index:IndexRangeScan
│ └─IndexRangeScan 1.00 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false
└─IndexJoin 0.22 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a)
├─IndexReader(Build) 0.45 root index:IndexRangeScan
│ └─IndexRangeScan 0.45 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false
└─TableReader(Probe) 1.00 root data:TableRangeScan
└─TableRangeScan 1.00 cop[tikv] table:t1 range: decided by [test.t.a], keep order:false
drop table if exists t;
Expand Down
7 changes: 7 additions & 0 deletions cmd/explaintest/t/cte.test
Original file line number Diff line number Diff line change
Expand Up @@ -168,3 +168,10 @@ create table t1(c1 bigint unsigned);
insert into t1 values(0);
--error 1690
with recursive cte1 as (select c1 - 1 c1 from t1 union all select c1 - 1 c1 from cte1 where c1 != 0) select * from cte1 dt1, cte1 dt2;
# case 32
drop table if exists t;
create table t(a int, b int, key (b));
desc with cte as (select * from t) select * from cte;
create SESSION binding for with cte as (select * from t) select * from cte using with cte as (select * from t use index(b)) select * from cte;
desc with cte as (select * from t) select * from cte;
desc with cte as (select * from t use index()) select * from cte;
4 changes: 3 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,9 @@ import (
zaplog "github.com/pingcap/log"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
tikvcfg "github.com/pingcap/tidb/store/tikv/config"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/versioninfo"
tikvcfg "github.com/tikv/client-go/v2/config"
tracing "github.com/uber/jaeger-client-go/config"
"go.uber.org/zap"
)
Expand Down Expand Up @@ -424,6 +424,7 @@ type Performance struct {
MemProfileInterval string `toml:"mem-profile-interval" json:"mem-profile-interval"`
IndexUsageSyncLease string `toml:"index-usage-sync-lease" json:"index-usage-sync-lease"`
GOGC int `toml:"gogc" json:"gogc"`
EnforceMPP bool `toml:"enforce-mpp" json:"enforce-mpp"`
}

// PlanCache is the PlanCache section of the config.
Expand Down Expand Up @@ -623,6 +624,7 @@ var defaultConf = Config{
// TODO: set indexUsageSyncLease to 60s.
IndexUsageSyncLease: "0s",
GOGC: 100,
EnforceMPP: false,
},
ProxyProtocol: ProxyProtocol{
Networks: "",
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,9 @@ cluster-ssl-key = ""
# "plaintext" means encryption is disabled.
spilled-file-encryption-method = "plaintext"

# Security Enhanced Mode (SEM) restricts the "SUPER" privilege and requires fine-grained privileges instead.
enable-sem = false

[status]
# If enable status report HTTP service.
report-status = true
Expand Down
2 changes: 1 addition & 1 deletion config/config_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (

"github.com/BurntSushi/toml"
"github.com/pingcap/errors"
tikvcfg "github.com/pingcap/tidb/store/tikv/config"
tikvcfg "github.com/tikv/client-go/v2/config"
)

// CloneConf deeply clones this config.
Expand Down
2 changes: 1 addition & 1 deletion ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,12 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/copr"
"github.com/pingcap/tidb/store/driver/backoff"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
decoder "github.com/pingcap/tidb/util/rowDecoder"
"github.com/tikv/client-go/v2/tikv"
"go.uber.org/zap"
)

Expand Down
48 changes: 40 additions & 8 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -850,7 +850,7 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol))
if mysql.HasPriKeyFlag(oldCol.Flag) {
job.State = model.JobStateCancelled
msg := "tidb_enable_change_column_type is true and this column has primary key flag"
msg := "this column has primary key flag"
return ver, errUnsupportedModifyColumn.GenWithStackByArgs(msg)
}

Expand All @@ -861,12 +861,15 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
// Since column type change is implemented as adding a new column then substituting the old one.
// Case exists when update-where statement fetch a NULL for not-null column without any default data,
// it will errors.
// So we set zero original default value here to prevent this error. besides, in insert & update records,
// So we set zero original default value here to prevent this error. Besides, in insert & update records,
// we have already implement using the casted value of relative column to insert rather than the origin
// default value.
originDefVal, err := generateOriginDefaultValue(jobParam.newCol)
if err != nil {
return ver, errors.Trace(err)
originDefVal := oldCol.GetOriginDefaultValue()
if originDefVal == nil {
originDefVal, err = generateOriginDefaultValue(jobParam.newCol)
if err != nil {
return ver, errors.Trace(err)
}
}
if err = jobParam.changingCol.SetOriginDefaultValue(originDefVal); err != nil {
return ver, errors.Trace(err)
Expand All @@ -884,8 +887,13 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
newIdxInfo := idxInfo.Clone()
newIdxInfo.Name = model.NewCIStr(genChangingIndexUniqueName(tblInfo, idxInfo))
newIdxInfo.ID = allocateIndexID(tblInfo)
newIdxInfo.Columns[offsets[i]].Name = newColName
newIdxInfo.Columns[offsets[i]].Offset = jobParam.changingCol.Offset
newIdxChangingCol := newIdxInfo.Columns[offsets[i]]
newIdxChangingCol.Name = newColName
newIdxChangingCol.Offset = jobParam.changingCol.Offset
canPrefix := types.IsTypePrefixable(jobParam.changingCol.Tp)
if !canPrefix || (canPrefix && jobParam.changingCol.Flen < newIdxChangingCol.Length) {
newIdxChangingCol.Length = types.UnspecifiedLength
}
jobParam.changingIdxs = append(jobParam.changingIdxs, newIdxInfo)
}
tblInfo.Indices = append(tblInfo.Indices, jobParam.changingIdxs...)
Expand Down Expand Up @@ -1044,10 +1052,11 @@ func (w *worker) doModifyColumnTypeWithData(
if err1 := t.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil {
logutil.BgLogger().Warn("[ddl] run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback",
zap.String("job", job.String()), zap.Error(err1))
return ver, errors.Trace(err)
}
logutil.BgLogger().Warn("[ddl] run modify column job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err))
job.State = model.JobStateRollingback
// Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs.
w.reorgCtx.cleanNotifyReorgCancel()
return ver, errors.Trace(err)
}
// Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs.
Expand All @@ -1070,6 +1079,13 @@ func (w *worker) doModifyColumnTypeWithData(
changingColumnUniqueName := changingCol.Name
changingCol.Name = colName
changingCol.ChangeStateInfo = nil
// After changing the column, the column's type is change, so it needs to set OriginDefaultValue back
// so that there is no error in getting the default value from OriginDefaultValue.
// Besides, nil data that was not backfilled in the "add column" is backfilled after the column is changed.
// So it can set OriginDefaultValue to nil.
if err = changingCol.SetOriginDefaultValue(nil); err != nil {
return ver, errors.Trace(err)
}
tblInfo.Indices = tblInfo.Indices[:len(tblInfo.Indices)-len(changingIdxs)]
// Adjust table column offset.
if err = adjustColumnInfoInModifyColumn(job, tblInfo, changingCol, oldCol, pos, changingColumnUniqueName.L); err != nil {
Expand Down Expand Up @@ -1109,8 +1125,24 @@ func (w *worker) updatePhysicalTableRow(t table.PhysicalTable, oldColInfo, colIn
return w.writePhysicalTableRecord(t.(table.PhysicalTable), typeUpdateColumnWorker, nil, oldColInfo, colInfo, reorgInfo)
}

// TestReorgGoroutineRunning is only used in test to indicate the reorg goroutine has been started.
var TestReorgGoroutineRunning = make(chan interface{})

// updateColumnAndIndexes handles the modify column reorganization state for a table.
func (w *worker) updateColumnAndIndexes(t table.Table, oldCol, col *model.ColumnInfo, idxes []*model.IndexInfo, reorgInfo *reorgInfo) error {
failpoint.Inject("mockInfiniteReorgLogic", func(val failpoint.Value) {
if val.(bool) {
a := new(interface{})
TestReorgGoroutineRunning <- a
for {
time.Sleep(30 * time.Millisecond)
if w.reorgCtx.isReorgCanceled() {
// Job is cancelled. So it can't be done.
failpoint.Return(errCancelledDDLJob)
}
}
}
})
// TODO: Support partition tables.
if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) {
err := w.updatePhysicalTableRow(t.(table.PhysicalTable), oldCol, col, reorgInfo)
Expand Down
15 changes: 8 additions & 7 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1159,6 +1159,7 @@ func (s *testColumnSuite) TestModifyColumn(c *C) {
WithLease(testLease),
)
ctx := testNewContext(d)

defer func() {
err := d.Stop()
c.Assert(err, IsNil)
Expand All @@ -1169,19 +1170,19 @@ func (s *testColumnSuite) TestModifyColumn(c *C) {
err error
}{
{"int", "bigint", nil},
{"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("can't change unsigned integer to signed or vice versa, and tidb_enable_change_column_type is false")},
{"int", "int unsigned", nil},
{"varchar(10)", "text", nil},
{"varbinary(10)", "blob", nil},
{"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")},
{"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10, and tidb_enable_change_column_type is false")},
{"varchar(10)", "varchar(8)", nil},
{"varchar(10)", "varchar(11)", nil},
{"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", nil},
{"decimal(2,1)", "decimal(3,2)", errUnsupportedModifyColumn.GenWithStackByArgs("decimal change from decimal(2, 1) to decimal(3, 2), and tidb_enable_change_column_type is false")},
{"decimal(2,1)", "decimal(2,2)", errUnsupportedModifyColumn.GenWithStackByArgs("decimal change from decimal(2, 1) to decimal(2, 2), and tidb_enable_change_column_type is false")},
{"decimal(2,1)", "decimal(3,2)", nil},
{"decimal(2,1)", "decimal(2,2)", nil},
{"decimal(2,1)", "decimal(2,1)", nil},
{"decimal(2,1)", "int", errUnsupportedModifyColumn.GenWithStackByArgs("type int(11) not match origin decimal(2,1), and tidb_enable_change_column_type is false")},
{"decimal", "int", errUnsupportedModifyColumn.GenWithStackByArgs("type int(11) not match origin decimal(10,0), and tidb_enable_change_column_type is false")},
{"decimal(2,1)", "bigint", errUnsupportedModifyColumn.GenWithStackByArgs("type bigint(20) not match origin decimal(2,1), and tidb_enable_change_column_type is false")},
{"decimal(2,1)", "int", nil},
{"decimal", "int", nil},
{"decimal(2,1)", "bigint", nil},
}
for _, tt := range tests {
ftA := s.colDefStrToFieldType(c, tt.origin)
Expand Down
Loading

0 comments on commit 87caedb

Please sign in to comment.