Skip to content

Commit

Permalink
Merge branch 'master' into foreyes_dev
Browse files Browse the repository at this point in the history
  • Loading branch information
foreyes authored Jul 16, 2019
2 parents 277a20e + 7403ce3 commit 943444a
Show file tree
Hide file tree
Showing 38 changed files with 649 additions and 545 deletions.
11 changes: 4 additions & 7 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -620,12 +620,9 @@ func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild
if !ok {
return
}
req := &tikvrpc.Request{
Type: tikvrpc.CmdDebugGetRegionProperties,
DebugGetRegionProperties: &debugpb.GetRegionPropertiesRequest{
RegionId: loc.Region.GetID(),
},
}
req := tikvrpc.NewRequest(tikvrpc.CmdDebugGetRegionProperties, &debugpb.GetRegionPropertiesRequest{
RegionId: loc.Region.GetID(),
})
var resp *tikvrpc.Response
var rpcCtx *tikv.RPCContext
rpcCtx, *err = e.cache.GetRPCContext(bo, loc.Region)
Expand Down Expand Up @@ -1002,7 +999,7 @@ func (e *AnalyzeFastExec) buildColumnStats(ID int64, collector *statistics.Sampl
collector.NullCount++
continue
}
bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, sample.Value)
bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, nil, sample.Value)
if err != nil {
return nil, nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -400,7 +400,7 @@ func (c *regionProperityClient) SendRequest(ctx context.Context, addr string, re
defer c.mu.Unlock()
c.mu.count++
// Mock failure once.
if req.DebugGetRegionProperties.RegionId == c.mu.regionID {
if req.DebugGetRegionProperties().RegionId == c.mu.regionID {
c.mu.regionID = 0
return &tikvrpc.Response{}, nil
}
Expand Down
44 changes: 32 additions & 12 deletions executor/checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,27 +187,47 @@ func newChecksumContext(db *model.DBInfo, table *model.TableInfo, startTs uint64
}

func (c *checksumContext) BuildRequests(ctx sessionctx.Context) ([]*kv.Request, error) {
reqs := make([]*kv.Request, 0, len(c.TableInfo.Indices)+1)
req, err := c.buildTableRequest(ctx)
if err != nil {
var partDefs []model.PartitionDefinition
if part := c.TableInfo.Partition; part != nil {
partDefs = part.Definitions
}

reqs := make([]*kv.Request, 0, (len(c.TableInfo.Indices)+1)*(len(partDefs)+1))
if err := c.appendRequest(ctx, c.TableInfo.ID, &reqs); err != nil {
return nil, err
}
reqs = append(reqs, req)

for _, partDef := range partDefs {
if err := c.appendRequest(ctx, partDef.ID, &reqs); err != nil {
return nil, err
}
}

return reqs, nil
}

func (c *checksumContext) appendRequest(ctx sessionctx.Context, tableID int64, reqs *[]*kv.Request) error {
req, err := c.buildTableRequest(ctx, tableID)
if err != nil {
return err
}

*reqs = append(*reqs, req)
for _, indexInfo := range c.TableInfo.Indices {
if indexInfo.State != model.StatePublic {
continue
}
req, err = c.buildIndexRequest(ctx, indexInfo)
req, err = c.buildIndexRequest(ctx, tableID, indexInfo)
if err != nil {
return nil, err
return err
}
reqs = append(reqs, req)
*reqs = append(*reqs, req)
}

return reqs, nil
return nil
}

func (c *checksumContext) buildTableRequest(ctx sessionctx.Context) (*kv.Request, error) {
func (c *checksumContext) buildTableRequest(ctx sessionctx.Context, tableID int64) (*kv.Request, error) {
checksum := &tipb.ChecksumRequest{
StartTs: c.StartTs,
ScanOn: tipb.ChecksumScanOn_Table,
Expand All @@ -217,13 +237,13 @@ func (c *checksumContext) buildTableRequest(ctx sessionctx.Context) (*kv.Request
ranges := ranger.FullIntRange(false)

var builder distsql.RequestBuilder
return builder.SetTableRanges(c.TableInfo.ID, ranges, nil).
return builder.SetTableRanges(tableID, ranges, nil).
SetChecksumRequest(checksum).
SetConcurrency(ctx.GetSessionVars().DistSQLScanConcurrency).
Build()
}

func (c *checksumContext) buildIndexRequest(ctx sessionctx.Context, indexInfo *model.IndexInfo) (*kv.Request, error) {
func (c *checksumContext) buildIndexRequest(ctx sessionctx.Context, tableID int64, indexInfo *model.IndexInfo) (*kv.Request, error) {
checksum := &tipb.ChecksumRequest{
StartTs: c.StartTs,
ScanOn: tipb.ChecksumScanOn_Index,
Expand All @@ -233,7 +253,7 @@ func (c *checksumContext) buildIndexRequest(ctx sessionctx.Context, indexInfo *m
ranges := ranger.FullRange()

var builder distsql.RequestBuilder
return builder.SetIndexRanges(ctx.GetSessionVars().StmtCtx, c.TableInfo.ID, indexInfo.ID, ranges).
return builder.SetIndexRanges(ctx.GetSessionVars().StmtCtx, tableID, indexInfo.ID, ranges).
SetChecksumRequest(checksum).
SetConcurrency(ctx.GetSessionVars().DistSQLScanConcurrency).
Build()
Expand Down
11 changes: 11 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -412,6 +412,17 @@ func (s *testSuite) TestAdmin(c *C) {
c.Assert(historyJobs, DeepEquals, historyJobs2)
}

func (s *testSuite) TestAdminChecksumOfPartitionedTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("USE test;")
tk.MustExec("DROP TABLE IF EXISTS admin_checksum_partition_test;")
tk.MustExec("CREATE TABLE admin_checksum_partition_test (a INT) PARTITION BY HASH(a) PARTITIONS 4;")
tk.MustExec("INSERT INTO admin_checksum_partition_test VALUES (1), (2);")

r := tk.MustQuery("ADMIN CHECKSUM TABLE admin_checksum_partition_test;")
r.Check(testkit.Rows("test admin_checksum_partition_test 1 5 5"))
}

func (s *testSuite) fillData(tk *testkit.TestKit, table string) {
tk.MustExec("use test")
tk.MustExec(fmt.Sprintf("create table %s(id int not null default 1, name varchar(255), PRIMARY KEY(id));", table))
Expand Down
20 changes: 18 additions & 2 deletions executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,9 +429,25 @@ func (s *testSuite2) TestValidateSetVar(c *C) {
c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err))

tk.MustExec("set @@tidb_batch_delete='On';")
tk.MustQuery("select @@tidb_batch_delete;").Check(testkit.Rows("1"))
tk.MustExec("set @@tidb_batch_delete='oFf';")
tk.MustQuery("select @@tidb_batch_delete;").Check(testkit.Rows("0"))
tk.MustExec("set @@tidb_batch_delete=1;")
tk.MustQuery("select @@tidb_batch_delete;").Check(testkit.Rows("1"))
tk.MustExec("set @@tidb_batch_delete=0;")
tk.MustQuery("select @@tidb_batch_delete;").Check(testkit.Rows("0"))

tk.MustExec("set @@tidb_opt_agg_push_down=off;")
tk.MustQuery("select @@tidb_opt_agg_push_down;").Check(testkit.Rows("0"))

tk.MustExec("set @@tidb_constraint_check_in_place=on;")
tk.MustQuery("select @@tidb_constraint_check_in_place;").Check(testkit.Rows("1"))

tk.MustExec("set @@tidb_general_log=0;")
tk.MustQuery("select @@tidb_general_log;").Check(testkit.Rows("0"))

tk.MustExec("set @@tidb_enable_streaming=1;")
tk.MustQuery("select @@tidb_enable_streaming;").Check(testkit.Rows("1"))

_, err = tk.Exec("set @@tidb_batch_delete=3;")
c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err))
Expand Down Expand Up @@ -789,9 +805,9 @@ func (s *testSuite2) TestEnableNoopFunctionsVar(c *C) {
_, err = tk.Exec(`set tidb_enable_noop_functions=11`)
c.Assert(err, NotNil)
tk.MustExec(`set tidb_enable_noop_functions="off";`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("off"))
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
tk.MustExec(`set tidb_enable_noop_functions="on";`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("on"))
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("1"))
tk.MustExec(`set tidb_enable_noop_functions=0;`)
tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0"))
}
2 changes: 1 addition & 1 deletion executor/show_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func (e *ShowExec) appendTableForStatsHistograms(dbName, tblName, partitionName
if col.IsInvalid(nil, false) {
continue
}
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count))
e.histogramToRow(dbName, tblName, partitionName, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count, false))
}
for _, idx := range statsTbl.Indices {
e.histogramToRow(dbName, tblName, partitionName, idx.Info.Name.O, 1, idx.Histogram, 0)
Expand Down
4 changes: 4 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4130,6 +4130,10 @@ func (s *testIntegrationSuite) TestFuncNameConst(c *C) {
r.Check(testkit.Rows("2"))
r = tk.MustQuery("SELECT concat('hello', name_const('test_string', 'world')) FROM t;")
r.Check(testkit.Rows("helloworld"))
r = tk.MustQuery("SELECT NAME_CONST('come', -1);")
r.Check(testkit.Rows("-1"))
r = tk.MustQuery("SELECT NAME_CONST('come', -1.0);")
r.Check(testkit.Rows("-1.0"))
err := tk.ExecToErr(`select name_const(a,b) from t;`)
c.Assert(err.Error(), Equals, "[planner:1210]Incorrect arguments to NAME_CONST")
err = tk.ExecToErr(`select name_const(a,"hello") from t;`)
Expand Down
8 changes: 4 additions & 4 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,22 +143,22 @@ func (s *testTableSuite) TestDataForTableStatsField(c *C) {
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("3 17 51 3"))
testkit.Rows("3 18 54 6"))
tk.MustExec(`insert into t(c, d, e) values(4, 5, "f")`)
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("4 17 68 4"))
testkit.Rows("4 18 72 8"))
tk.MustExec("delete from t where c >= 3")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("2 17 34 2"))
testkit.Rows("2 18 36 4"))
tk.MustExec("delete from t where c=3")
c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil)
c.Assert(h.Update(is), IsNil)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("2 17 34 2"))
testkit.Rows("2 18 36 4"))
}

func (s *testTableSuite) TestCharacterSetCollations(c *C) {
Expand Down
15 changes: 15 additions & 0 deletions planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1090,3 +1090,18 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) {
" └─TableScan_19 6.00 cop table:t, keep order:false",
))
}

func (s *testAnalyzeSuite) TestUpdateProjEliminate(c *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("drop table if exists t")
tk.MustExec("create table t(a int, b int)")
tk.MustExec("explain update t t1, (select distinct b from t) t2 set t1.b = t2.b")
}
4 changes: 3 additions & 1 deletion planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2585,7 +2585,9 @@ func (b *PlanBuilder) buildUpdate(update *ast.UpdateStmt) (Plan, error) {

updt := Update{OrderedList: orderedList}.Init(b.ctx)
updt.SetSchema(p.Schema())
updt.SelectPlan, err = DoOptimize(b.optFlag, p)
// We cannot apply projection elimination when building the subplan, because
// columns in orderedList cannot be resolved.
updt.SelectPlan, err = DoOptimize(b.optFlag&^flagEliminateProjection, p)
if err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/plan_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func SetPBColumnsDefaultValue(ctx sessionctx.Context, pbColumns []*tipb.ColumnIn
return err
}

pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx, d)
pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(sessVars.StmtCtx, nil, d)
if err != nil {
return err
}
Expand Down
7 changes: 6 additions & 1 deletion planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,12 @@ func (p *preprocessor) Leave(in ast.Node) (out ast.Node, ok bool) {
p.err = expression.ErrIncorrectParameterCount.GenWithStackByArgs(x.FnName.L)
} else {
_, isValueExpr1 := x.Args[0].(*driver.ValueExpr)
_, isValueExpr2 := x.Args[1].(*driver.ValueExpr)
isValueExpr2 := false
switch x.Args[1].(type) {
case *driver.ValueExpr, *ast.UnaryOperationExpr:
isValueExpr2 = true
}

if !isValueExpr1 || !isValueExpr2 {
p.err = ErrWrongArguments.GenWithStackByArgs("NAME_CONST")
}
Expand Down
9 changes: 3 additions & 6 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,12 +151,9 @@ func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey []by
return nil, errors.Trace(err)
}

tikvReq := &tikvrpc.Request{
Type: tikvrpc.CmdMvccGetByStartTs,
MvccGetByStartTs: &kvrpcpb.MvccGetByStartTsRequest{
StartTs: startTS,
},
}
tikvReq := tikvrpc.NewRequest(tikvrpc.CmdMvccGetByStartTs, &kvrpcpb.MvccGetByStartTsRequest{
StartTs: startTS,
})
tikvReq.Context.Priority = kvrpcpb.CommandPri_Low
kvResp, err := t.Store.SendReq(bo, tikvReq, curRegion.Region, time.Hour)
if err != nil {
Expand Down
21 changes: 9 additions & 12 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -374,9 +374,16 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string,
return "1", nil
}
return value, ErrWrongValueForVar.GenWithStackByArgs(name, value)
case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin,
case TiDBSkipUTF8Check, TiDBOptAggPushDown,
TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze,
TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming,
TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction,
TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs,
TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace:
fallthrough
case GeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin,
CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates,
SkipNameResolve, SQLSafeUpdates, TiDBConstraintCheckInPlace, serverReadOnly, SlaveAllowBatching,
SkipNameResolve, SQLSafeUpdates, serverReadOnly, SlaveAllowBatching,
Flush, PerformanceSchema, LocalInFile, ShowOldTemporals, KeepFilesOnCreate, AutoCommit,
SQLWarnings, UniqueChecks, OldAlterTable, LogBinTrustFunctionCreators, SQLBigSelects,
BinlogDirectNonTransactionalUpdates, SQLQuoteShowCreate, AutomaticSpPrivileges,
Expand Down Expand Up @@ -415,16 +422,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string,
}
}
return value, ErrWrongValueForVar.GenWithStackByArgs(name, value)
case TiDBSkipUTF8Check, TiDBOptAggPushDown,
TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze,
TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming,
TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction,
TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs,
TiDBScatterRegion:
if strings.EqualFold(value, "ON") || value == "1" || strings.EqualFold(value, "OFF") || value == "0" {
return value, nil
}
return value, ErrWrongValueForVar.GenWithStackByArgs(name, value)
case MaxExecutionTime:
return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars)
case TiDBEnableTablePartition:
Expand Down
4 changes: 2 additions & 2 deletions statistics/handle/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ func (s *testStatsSuite) TestDDLHistogram(c *C) {
tableInfo = tbl.Meta()
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Pseudo, IsFalse)
c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count), Equals, 3.0)
c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)

testKit.MustExec("create index i on t(c2, c1)")
testKit.MustExec("analyze table t")
Expand Down Expand Up @@ -212,6 +212,6 @@ PARTITION BY RANGE ( a ) (
for _, def := range pi.Definitions {
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
c.Assert(statsTbl.Pseudo, IsFalse)
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 3.0)
c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 3.0)
}
}
16 changes: 8 additions & 8 deletions statistics/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,19 +208,19 @@ func (s *testStatsSuite) TestAvgColLen(c *C) {
c.Assert(err, IsNil)
tableInfo := tbl.Meta()
statsTbl := do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.0)

// The size of varchar type is LEN + BYTE, here is 1 + 7 = 8
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29')")
testKit.MustExec("analyze table t")
statsTbl = do.StatsHandle().GetTableStats(tableInfo)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 10.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 10.5)
c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0)
}

func (s *testStatsSuite) TestDurationToTS(c *C) {
Expand Down
Loading

0 comments on commit 943444a

Please sign in to comment.