Skip to content

Commit

Permalink
Merge branch 'master' into issue-23036
Browse files Browse the repository at this point in the history
  • Loading branch information
guo-shaoge authored Mar 4, 2021
2 parents 7bfc7bc + 5f73c82 commit faadae1
Show file tree
Hide file tree
Showing 123 changed files with 2,343 additions and 859 deletions.
32 changes: 22 additions & 10 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,10 @@ import (
func TestT(t *testing.T) {
CustomVerboseFlag = true
logLevel := os.Getenv("log_level")
logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
if err != nil {
t.Fatal(err)
}
autoid.SetStep(5000)
TestingT(t)
}
Expand Down Expand Up @@ -375,9 +378,11 @@ func (s *testSuite) TestGlobalBinding(c *C) {
}

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, testSQL.memoryUsage)

sql, hash := normalizeWithDefaultDB(c, testSQL.querySQL, "test")
Expand Down Expand Up @@ -432,9 +437,11 @@ func (s *testSuite) TestGlobalBinding(c *C) {
bindData = s.domain.BindHandle().GetBindRecord(hash, sql, "test")
c.Check(bindData, IsNil)

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
// From newly created global bind handle.
c.Assert(pb.GetGauge().GetValue(), Equals, testSQL.memoryUsage)

Expand Down Expand Up @@ -482,9 +489,11 @@ func (s *testSuite) TestSessionBinding(c *C) {
}

pb := &dto.Metric{}
metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(1))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, testSQL.memoryUsage)

handle := tk.Se.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
Expand Down Expand Up @@ -530,9 +539,11 @@ func (s *testSuite) TestSessionBinding(c *C) {
c.Check(bindData.OriginalSQL, Equals, testSQL.originSQL)
c.Check(len(bindData.Bindings), Equals, 0)

metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeSession, bindinfo.Using).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetGauge().GetValue(), Equals, float64(0))
}
}
Expand All @@ -554,7 +565,8 @@ func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) {
metrics.BindUsageCounter.Reset()
c.Assert(tk.HasPlan("SELECT * from t1,t2 where t1.id = t2.id", "MergeJoin"), IsTrue)
pb := &dto.Metric{}
metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Write(pb)
err := metrics.BindUsageCounter.WithLabelValues(metrics.ScopeGlobal).Write(pb)
c.Assert(err, IsNil)
c.Assert(pb.GetCounter().GetValue(), Equals, float64(1))

// Test 'tidb_use_plan_baselines'
Expand Down
46 changes: 20 additions & 26 deletions cmd/explaintest/r/clustered_index.result
Original file line number Diff line number Diff line change
Expand Up @@ -28,11 +28,10 @@ load stats 's/wout_cluster_index_tbl_3.json';
load stats 's/wout_cluster_index_tbl_4.json';
explain select count(*) from with_cluster_index.tbl_0 where col_0 < 5429 ;
id estRows task access object operator info
StreamAgg_27 1.00 root funcs:count(Column#9)->Column#6
└─TableReader_28 1.00 root data:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9
└─Selection_26 798.90 cop[tikv] lt(with_cluster_index.tbl_0.col_0, 5429)
└─TableFullScan_25 2244.00 cop[tikv] table:tbl_0 keep order:false
StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6
└─IndexReader_18 1.00 root index:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8
└─IndexRangeScan_16 798.90 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,5429), keep order:false
explain select count(*) from wout_cluster_index.tbl_0 where col_0 < 5429 ;
id estRows task access object operator info
StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7
Expand All @@ -41,11 +40,10 @@ StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7
└─IndexRangeScan_16 798.90 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,5429), keep order:false
explain select count(*) from with_cluster_index.tbl_0 where col_0 < 41 ;
id estRows task access object operator info
HashAgg_17 1.00 root funcs:count(Column#8)->Column#6
└─IndexLookUp_18 1.00 root
├─IndexRangeScan_15(Build) 41.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,41), keep order:false
└─HashAgg_7(Probe) 1.00 cop[tikv] funcs:count(1)->Column#8
└─TableRowIDScan_16 41.00 cop[tikv] table:tbl_0 keep order:false
StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6
└─IndexReader_18 1.00 root index:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8
└─IndexRangeScan_16 41.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,41), keep order:false
explain select count(*) from wout_cluster_index.tbl_0 where col_0 < 41 ;
id estRows task access object operator info
StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7
Expand Down Expand Up @@ -78,19 +76,16 @@ StreamAgg_37 1.00 root funcs:sum(Column#20)->Column#7
└─TableFullScan_35 2244.00 cop[tikv] table:tbl_0 keep order:false
explain select col_0 from with_cluster_index.tbl_0 where col_0 <= 0 ;
id estRows task access object operator info
Projection_4 1.00 root with_cluster_index.tbl_0.col_0
└─IndexLookUp_10 1.00 root
├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tbl_0 keep order:false
IndexReader_6 1.00 root index:IndexRangeScan_5
└─IndexRangeScan_5 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
explain select col_0 from wout_cluster_index.tbl_0 where col_0 <= 0 ;
id estRows task access object operator info
IndexReader_6 1.00 root index:IndexRangeScan_5
└─IndexRangeScan_5 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
explain select col_3 from with_cluster_index.tbl_0 where col_3 >= '1981-09-15' ;
id estRows task access object operator info
Projection_4 1859.31 root with_cluster_index.tbl_0.col_3
└─TableReader_6 1859.31 root data:TableRangeScan_5
└─TableRangeScan_5 1859.31 cop[tikv] table:tbl_0 range:[1981-09-15 00:00:00,+inf], keep order:false
TableReader_6 1859.31 root data:TableRangeScan_5
└─TableRangeScan_5 1859.31 cop[tikv] table:tbl_0 range:[1981-09-15 00:00:00,+inf], keep order:false
explain select col_3 from wout_cluster_index.tbl_0 where col_3 >= '1981-09-15' ;
id estRows task access object operator info
IndexReader_10 1859.31 root index:IndexRangeScan_9
Expand All @@ -112,10 +107,10 @@ HashJoin_22 2533.51 root right outer join, equal:[eq(wout_cluster_index.tbl_2.c
└─TableFullScan_41 4673.00 cop[tikv] table:tbl_2 keep order:false
explain select count(*) from with_cluster_index.tbl_0 where col_0 <= 0 ;
id estRows task access object operator info
StreamAgg_10 1.00 root funcs:count(1)->Column#6
└─IndexLookUp_24 1.00 root
├─IndexRangeScan_22(Build) 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
└─TableRowIDScan_23(Probe) 1.00 cop[tikv] table:tbl_0 keep order:false
StreamAgg_16 1.00 root funcs:count(Column#8)->Column#6
└─IndexReader_17 1.00 root index:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8
└─IndexRangeScan_11 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
explain select count(*) from wout_cluster_index.tbl_0 where col_0 <= 0 ;
id estRows task access object operator info
StreamAgg_16 1.00 root funcs:count(Column#9)->Column#7
Expand All @@ -124,11 +119,10 @@ StreamAgg_16 1.00 root funcs:count(Column#9)->Column#7
└─IndexRangeScan_11 1.00 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[-inf,0], keep order:false
explain select count(*) from with_cluster_index.tbl_0 where col_0 >= 803163 ;
id estRows task access object operator info
HashAgg_17 1.00 root funcs:count(Column#8)->Column#6
└─IndexLookUp_18 1.00 root
├─IndexRangeScan_15(Build) 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false
└─HashAgg_7(Probe) 1.00 cop[tikv] funcs:count(1)->Column#8
└─TableRowIDScan_16 109.70 cop[tikv] table:tbl_0 keep order:false
StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6
└─IndexReader_18 1.00 root index:StreamAgg_9
└─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8
└─IndexRangeScan_16 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false
explain select count(*) from wout_cluster_index.tbl_0 where col_0 >= 803163 ;
id estRows task access object operator info
StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7
Expand Down
5 changes: 2 additions & 3 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1928,8 +1928,7 @@ func checkGlobalIndexRow(c *C, ctx sessionctx.Context, tblInfo *model.TableInfo,
c.Assert(err, IsNil)
value, err := txn.Get(context.Background(), key)
c.Assert(err, IsNil)
colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns),
tablecodec.HandleDefault, idxColInfos)
colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns), tablecodec.HandleDefault, idxColInfos)
c.Assert(err, IsNil)
c.Assert(colVals, HasLen, len(idxVals)+2)
for i, val := range idxVals {
Expand Down Expand Up @@ -4107,7 +4106,7 @@ func (s *testSerialDBSuite) TestModifyColumnBetweenStringTypes(c *C) {
c.Assert(c2.FieldType.Tp, Equals, mysql.TypeBlob)

// text to set
tk.MustGetErrMsg("alter table tt change a a set('111', '2222');", "[types:1265]Data truncated for column 'a', value is 'KindBytes 10000'")
tk.MustGetErrMsg("alter table tt change a a set('111', '2222');", "[types:1265]Data truncated for column 'a', value is 'KindString 10000'")
tk.MustExec("alter table tt change a a set('111', '10000');")
c2 = getModifyColumn(c, s.s.(sessionctx.Context), "test", "tt", "a", false)
c.Assert(c2.FieldType.Tp, Equals, mysql.TypeSet)
Expand Down
9 changes: 9 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1438,6 +1438,9 @@ func buildTableInfo(
tbInfo.PKIsHandle = true
} else {
tbInfo.IsCommonHandle = noBinlog
if tbInfo.IsCommonHandle {
tbInfo.CommonHandleVersion = 1
}
if !noBinlog {
errMsg := "cannot build clustered index table because the binlog is ON"
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf(errMsg))
Expand All @@ -1449,6 +1452,9 @@ func buildTableInfo(
tbInfo.PKIsHandle = !alterPKConf
} else {
tbInfo.IsCommonHandle = !alterPKConf && ctx.GetSessionVars().EnableClusteredIndex && noBinlog
if tbInfo.IsCommonHandle {
tbInfo.CommonHandleVersion = 1
}
}
}
if tbInfo.PKIsHandle || tbInfo.IsCommonHandle {
Expand Down Expand Up @@ -4294,6 +4300,9 @@ func (d *ddl) AlterTableAddStatistics(ctx sessionctx.Context, ident ast.Ident, s
return err
}
tblInfo := tbl.Meta()
if tblInfo.GetPartitionInfo() != nil {
return errors.New("Extended statistics on partitioned tables are not supported now")
}
colIDs := make([]int64, 0, 2)
// Check whether columns exist.
for _, colName := range stats.Columns {
Expand Down
10 changes: 7 additions & 3 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -821,6 +821,7 @@ type indexRecord struct {
handle kv.Handle
key []byte // It's used to lock a record. Record it to reduce the encoding time.
vals []types.Datum // It's the index values.
rsData []types.Datum // It's the restored data for handle.
skip bool // skip indicates that the index key is already exists, we should not add it.
}

Expand Down Expand Up @@ -922,7 +923,9 @@ func (w *baseIndexWorker) getIndexRecord(idxInfo *model.IndexInfo, handle kv.Han
}
idxVal[j] = idxColumnVal
}
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal}

rsData := tables.TryGetHandleRestoredDataWrapper(w.table, nil, w.rowMap)
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal, rsData: rsData}
return idxRecord, nil
}

Expand Down Expand Up @@ -1128,7 +1131,8 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i
} else if w.distinctCheckFlags[i] {
// The keys in w.batchCheckKeys also maybe duplicate,
// so we need to backfill the not found key into `batchVals` map.
val, err := w.index.GenIndexValue(stmtCtx, idxRecords[i].vals, w.distinctCheckFlags[i], false, idxRecords[i].handle)
needRsData := tables.NeedRestoredData(w.index.Meta().Columns, w.table.Meta().Columns)
val, err := tablecodec.GenIndexValuePortal(stmtCtx, w.table.Meta(), w.index.Meta(), needRsData, w.distinctCheckFlags[i], false, idxRecords[i].vals, idxRecords[i].handle, 0, idxRecords[i].rsData)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -1185,7 +1189,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC
}

// Create the index.
handle, err := w.index.Create(w.sessCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle)
handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, idxRecord.handle, idxRecord.rsData)
if err != nil {
if kv.ErrKeyExists.Equal(err) && idxRecord.handle.Equal(handle) {
// Index already exists, skip it.
Expand Down
2 changes: 1 addition & 1 deletion ddl/reorg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ func (s *testDDLSuite) TestReorg(c *C) {
Job: job,
currElement: e,
}
mockTbl := tables.MockTableFromMeta(&model.TableInfo{IsCommonHandle: s.IsCommonHandle})
mockTbl := tables.MockTableFromMeta(&model.TableInfo{IsCommonHandle: s.IsCommonHandle, CommonHandleVersion: 1})
err = d.generalWorker().runReorgJob(m, rInfo, mockTbl.Meta(), d.lease, f)
c.Assert(err, NotNil)

Expand Down
5 changes: 4 additions & 1 deletion distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,10 @@ var _ = Suite(&testSuite{})
func TestT(t *testing.T) {
CustomVerboseFlag = true
logLevel := os.Getenv("log_level")
logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
if err != nil {
t.Fatal(err)
}
TestingT(t)
}

Expand Down
6 changes: 4 additions & 2 deletions executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,7 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err
type recoverRows struct {
handle kv.Handle
idxVals []types.Datum
rsData []types.Datum
skip bool
}

Expand Down Expand Up @@ -377,7 +378,8 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists
}
idxVals := extractIdxVals(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes, idxValLen)
e.idxValsBufs[result.scanRowCount] = idxVals
e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, skip: false})
rsData := tables.TryGetHandleRestoredDataWrapper(e.table, plannercore.GetCommonHandleDatum(e.handleCols, row), nil)
e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, rsData: rsData, skip: false})
result.scanRowCount++
result.currentHandle = handle
}
Expand Down Expand Up @@ -463,7 +465,7 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa
return result, err
}

_, err = e.index.Create(e.ctx, txn.GetUnionStore(), row.idxVals, row.handle)
_, err = e.index.Create(e.ctx, txn, row.idxVals, row.handle, row.rsData)
if err != nil {
return result, err
}
Expand Down
Loading

0 comments on commit faadae1

Please sign in to comment.