Skip to content

Commit

Permalink
Merge branch 'master' into stable-nodify
Browse files Browse the repository at this point in the history
  • Loading branch information
ngaut authored Mar 11, 2021
2 parents b239cd0 + 07b19e0 commit b8c5f03
Show file tree
Hide file tree
Showing 20 changed files with 194 additions and 49 deletions.
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -617,7 +617,7 @@ var defaultConf = Config{
HeaderTimeout: 5,
},
PreparedPlanCache: PreparedPlanCache{
Enabled: true,
Enabled: false,
Capacity: 100,
MemoryGuardRatio: 0.1,
},
Expand Down
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ networks = ""
header-timeout = 5

[prepared-plan-cache]
enabled = true
enabled = false
capacity = 100
memory-guard-ratio = 0.1

Expand Down
6 changes: 1 addition & 5 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -692,6 +692,7 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio
}

do.SchemaValidator = NewSchemaValidator(ddlLease, do)
do.expensiveQueryHandle = expensivequery.NewExpensiveQueryHandle(do.exit)
return do
}

Expand Down Expand Up @@ -1296,11 +1297,6 @@ func (do *Domain) ExpensiveQueryHandle() *expensivequery.Handle {
return do.expensiveQueryHandle
}

// InitExpensiveQueryHandle init the expensive query handler.
func (do *Domain) InitExpensiveQueryHandle() {
do.expensiveQueryHandle = expensivequery.NewExpensiveQueryHandle(do.exit)
}

const privilegeKey = "/tidb/privilege"

// NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches
Expand Down
2 changes: 1 addition & 1 deletion errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -991,7 +991,7 @@ const (
ErrInvalidTableSample = 8128
ErrJSONObjectKeyTooLong = 8129
ErrMultiStatementDisabled = 8130
ErrBuildGlobalLevelStatsFailed = 8131
ErrPartitionStatsMissing = 8131

// Error codes used by TiDB ddl package
ErrUnsupportedDDLOperation = 8200
Expand Down
4 changes: 2 additions & 2 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1032,8 +1032,8 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{

ErrInvalidTableSample: mysql.Message("Invalid TABLESAMPLE: %s", nil),

ErrJSONObjectKeyTooLong: mysql.Message("TiDB does not yet support JSON objects with the key length >= 65536", nil),
ErrBuildGlobalLevelStatsFailed: mysql.Message("Build table: %s global-level stats failed due to missing partition-level stats", nil),
ErrJSONObjectKeyTooLong: mysql.Message("TiDB does not yet support JSON objects with the key length >= 65536", nil),
ErrPartitionStatsMissing: mysql.Message("Build table: %s global-level stats failed due to missing partition-level stats", nil),

ErrInvalidPlacementSpec: mysql.Message("Invalid placement policy '%s': %s", nil),
ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil),
Expand Down
31 changes: 20 additions & 11 deletions executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,6 +281,11 @@ type topNRows struct {
currSize uint64
limitSize uint64
sepSize uint64
// If sep is truncated, we need to append part of sep to result.
// In the following example, session.group_concat_max_len is 10 and sep is '---'.
// ('---', 'ccc') should be poped from heap, so '-' should be appended to result.
// eg: 'aaa---bbb---ccc' -> 'aaa---bbb-'
isSepTruncated bool
}

func (h topNRows) Len() int {
Expand Down Expand Up @@ -349,6 +354,7 @@ func (h *topNRows) tryToAdd(row sortRow) (truncated bool, memDelta int64) {
memDelta -= GetDatumMemSize(dt)
}
heap.Pop(h)
h.isSepTruncated = true
}
}
return true, memDelta
Expand All @@ -369,10 +375,11 @@ func (h *topNRows) concat(sep string, truncated bool) string {
}
buffer.Write(row.buffer.Bytes())
}
if truncated && uint64(buffer.Len()) < h.limitSize {
// append the last separator, because the last separator may be truncated in tryToAdd.
if h.isSepTruncated {
buffer.WriteString(sep)
buffer.Truncate(int(h.limitSize))
if uint64(buffer.Len()) > h.limitSize {
buffer.Truncate(int(h.limitSize))
}
}
return buffer.String()
}
Expand Down Expand Up @@ -402,10 +409,11 @@ func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int6
}
p := &partialResult4GroupConcatOrder{
topN: &topNRows{
desc: desc,
currSize: 0,
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
desc: desc,
currSize: 0,
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
isSepTruncated: false,
},
}
return PartialResult(p), DefPartialResult4GroupConcatOrderSize + DefTopNRowsSize
Expand Down Expand Up @@ -504,10 +512,11 @@ func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDe
valSet, setSize := set.NewStringSetWithMemoryUsage()
p := &partialResult4GroupConcatOrderDistinct{
topN: &topNRows{
desc: desc,
currSize: 0,
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
desc: desc,
currSize: 0,
limitSize: e.maxLen,
sepSize: uint64(len(e.sep)),
isSepTruncated: false,
},
valSet: valSet,
}
Expand Down
17 changes: 17 additions & 0 deletions executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -641,6 +641,23 @@ func (s *testSuiteAgg) TestGroupConcatAggr(c *C) {

// issue #9920
tk.MustQuery("select group_concat(123, null)").Check(testkit.Rows("<nil>"))

// issue #23129
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(cid int, sname varchar(100));")
tk.MustExec("insert into t1 values(1, 'Bob'), (1, 'Alice');")
tk.MustExec("insert into t1 values(3, 'Ace');")
tk.MustExec("set @@group_concat_max_len=5;")
rows := tk.MustQuery("select group_concat(sname order by sname) from t1 group by cid;")
rows.Check(testkit.Rows("Alice", "Ace"))

tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t1(c1 varchar(10));")
tk.MustExec("insert into t1 values('0123456789');")
tk.MustExec("insert into t1 values('12345');")
tk.MustExec("set @@group_concat_max_len=8;")
rows = tk.MustQuery("select group_concat(c1 order by c1) from t1 group by c1;")
rows.Check(testkit.Rows("01234567", "12345"))
}

func (s *testSuiteAgg) TestSelectDistinct(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
for globalStatsID, info := range globalStatsMap {
globalStats, err := statsHandle.MergePartitionStats2GlobalStats(e.ctx, e.opts, infoschema.GetInfoSchema(e.ctx), globalStatsID.tableID, info.isIndex, info.idxID)
if err != nil {
if types.ErrBuildGlobalLevelStatsFailed.Equal(err) {
if types.ErrPartitionStatsMissing.Equal(err) {
// When we find some partition-level stats are missing, we need to report warning.
e.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
continue
Expand Down
14 changes: 12 additions & 2 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5893,9 +5893,19 @@ func (s *testIntegrationSuite) TestCastStrToInt(c *C) {
func (s *testIntegrationSerialSuite) TestPreparePlanCache(c *C) {
tk := testkit.NewTestKit(c, s.store)

// Plan cache should now be on by default
c.Assert(plannercore.PreparedPlanCacheEnabled(), Equals, true)
// Plan cache should now be off by default
c.Assert(plannercore.PreparedPlanCacheEnabled(), Equals, false)

orgEnable := plannercore.PreparedPlanCacheEnabled()
defer func() {
plannercore.SetPreparedPlanCache(orgEnable)
}()
plannercore.SetPreparedPlanCache(true)
var err error
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
})
c.Assert(err, IsNil)
// Use the example from the docs https://docs.pingcap.com/tidb/stable/sql-prepare-plan-cache
tk.MustExec("use test")
tk.MustExec("drop table if exists t;")
Expand Down
4 changes: 2 additions & 2 deletions planner/core/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@ import (

var (
// preparedPlanCacheEnabledValue stores the global config "prepared-plan-cache-enabled".
// The value is true unless "prepared-plan-cache-enabled" is FALSE in configuration.
preparedPlanCacheEnabledValue int32 = 1
// The value is false unless "prepared-plan-cache-enabled" is true in configuration.
preparedPlanCacheEnabledValue int32 = 0
// PreparedPlanCacheCapacity stores the global config "prepared-plan-cache-capacity".
PreparedPlanCacheCapacity uint = 100
// PreparedPlanCacheMemoryGuardRatio stores the global config "prepared-plan-cache-memory-guard-ratio".
Expand Down
26 changes: 26 additions & 0 deletions planner/core/partition_pruner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -449,6 +449,32 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) {
}
}

func (s *testPartitionPruneSuit) TestIssue22635(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("USE test;")
tk.MustExec("DROP TABLE IF EXISTS t1")
tk.MustExec(`
CREATE TABLE t1 (
a int(11) DEFAULT NULL,
b int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY HASH( a )
PARTITIONS 4`)
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows()) // work fine without any error

tk.MustExec("insert into t1 values (1, 1)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 1"))

tk.MustExec("insert into t1 values (2, 2), (2, 2)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 2"))

tk.MustExec("insert into t1 values (3, 3), (3, 3), (3, 3)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("1 3"))

tk.MustExec("insert into t1 values (4, 4), (4, 4), (4, 4), (4, 4)")
tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("4 4"))
}

func (s *testPartitionPruneSuit) TestIssue22898(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("USE test;")
Expand Down
42 changes: 41 additions & 1 deletion planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,8 +211,48 @@ func (s *partitionProcessor) pruneHashPartition(ctx sessionctx.Context, tbl tabl
return used, nil
}

// reconstructTableColNames reconstructs FieldsNames according to ds.TblCols.
// ds.names may not match ds.TblCols since ds.names is pruned while ds.TblCols contains all original columns.
// please see https://github.com/pingcap/tidb/issues/22635 for more details.
func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types.FieldName, error) {
names := make([]*types.FieldName, 0, len(ds.TblCols))
colsInfo := ds.table.FullHiddenColsAndVisibleCols()
colsInfoMap := make(map[int64]*table.Column, len(colsInfo))
for _, c := range colsInfo {
colsInfoMap[c.ID] = c
}
for _, colExpr := range ds.TblCols {
if colExpr.ID == model.ExtraHandleID {
names = append(names, &types.FieldName{
DBName: ds.DBName,
TblName: ds.tableInfo.Name,
ColName: model.ExtraHandleName,
OrigColName: model.ExtraHandleName,
})
continue
}
if colInfo, found := colsInfoMap[colExpr.ID]; found {
names = append(names, &types.FieldName{
DBName: ds.DBName,
TblName: ds.tableInfo.Name,
ColName: colInfo.Name,
OrigTblName: ds.tableInfo.Name,
OrigColName: colInfo.Name,
Hidden: colInfo.Hidden,
})
continue
}
return nil, errors.New(fmt.Sprintf("information of column %v is not found", colExpr.String()))
}
return names, nil
}

func (s *partitionProcessor) processHashPartition(ds *DataSource, pi *model.PartitionInfo) (LogicalPlan, error) {
used, err := s.pruneHashPartition(ds.SCtx(), ds.table, ds.partitionNames, ds.allConds, ds.TblCols, ds.names)
names, err := s.reconstructTableColNames(ds)
if err != nil {
return nil, err
}
used, err := s.pruneHashPartition(ds.SCtx(), ds.table, ds.partitionNames, ds.allConds, ds.TblCols, names)
if err != nil {
return nil, err
}
Expand Down
1 change: 0 additions & 1 deletion session/bootstrap_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,6 @@ func (s *testBootstrapSuite) TestBootstrapInitExpensiveQueryHandle(c *C) {
dom := domain.GetDomain(se)
c.Assert(dom, NotNil)
defer dom.Close()
dom.InitExpensiveQueryHandle()
c.Assert(dom.ExpensiveQueryHandle(), NotNil)
}

Expand Down
1 change: 0 additions & 1 deletion session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2172,7 +2172,6 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) {
}

dom := domain.GetDomain(se)
dom.InitExpensiveQueryHandle()

se2, err := createSession(store)
if err != nil {
Expand Down
18 changes: 14 additions & 4 deletions sessionctx/binloginfo/binloginfo.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,8 +44,8 @@ func init() {
// shared by all sessions.
var pumpsClient *pumpcli.PumpsClient
var pumpsClientLock sync.RWMutex
var shardPat = regexp.MustCompile(`SHARD_ROW_ID_BITS\s*=\s*\d+\s*`)
var preSplitPat = regexp.MustCompile(`PRE_SPLIT_REGIONS\s*=\s*\d+\s*`)
var shardPat = regexp.MustCompile(`(?P<REPLACE>SHARD_ROW_ID_BITS\s*=\s*\d+\s*)`)
var preSplitPat = regexp.MustCompile(`(?P<REPLACE>PRE_SPLIT_REGIONS\s*=\s*\d+\s*)`)

// BinlogInfo contains binlog data and binlog client.
type BinlogInfo struct {
Expand Down Expand Up @@ -300,7 +300,7 @@ func SetDDLBinlog(client *pumpcli.PumpsClient, txn kv.Transaction, jobID int64,
const specialPrefix = `/*T! `

// AddSpecialComment uses to add comment for table option in DDL query.
// Export for testing.
// Used by pingcap/ticdc.
func AddSpecialComment(ddlQuery string) string {
if strings.Contains(ddlQuery, specialPrefix) || strings.Contains(ddlQuery, driver.SpecialCommentVersionPrefix) {
return ddlQuery
Expand All @@ -320,7 +320,17 @@ func addSpecialCommentByRegexps(ddlQuery string, prefix string, regs ...*regexp.
minIdx := math.MaxInt64
for i := 0; i < len(regs); {
reg := regs[i]
loc := reg.FindStringIndex(upperQuery)
locs := reg.FindStringSubmatchIndex(upperQuery)
ns := reg.SubexpNames()
var loc []int
if len(locs) > 0 {
for i, n := range ns {
if n == "REPLACE" {
loc = locs[i*2 : (i+1)*2]
break
}
}
}
if len(loc) < 2 {
i++
continue
Expand Down
28 changes: 28 additions & 0 deletions sessionctx/binloginfo/binloginfo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -614,6 +614,18 @@ func (s *testBinlogSuite) TestAddSpecialComment(c *C) {
"create table t1 (id int, a varchar(255), primary key (a, b) clustered);",
"create table t1 (id int, a varchar(255), primary key (a, b) /*T![clustered_index] clustered */ );",
},
{
"create table t1(id int, v int, primary key(a) clustered);",
"create table t1(id int, v int, primary key(a) /*T![clustered_index] clustered */ );",
},
{
"create table t1(id int primary key clustered, v int);",
"create table t1(id int primary key /*T![clustered_index] clustered */ , v int);",
},
{
"alter table t add primary key(a) clustered;",
"alter table t add primary key(a) /*T![clustered_index] clustered */ ;",
},
{
"create table t1 (id int, a varchar(255), primary key (a, b) nonclustered);",
"create table t1 (id int, a varchar(255), primary key (a, b) /*T![clustered_index] nonclustered */ );",
Expand All @@ -622,6 +634,22 @@ func (s *testBinlogSuite) TestAddSpecialComment(c *C) {
"create table t1 (id int, a varchar(255), primary key (a, b) /*T![clustered_index] nonclustered */);",
"create table t1 (id int, a varchar(255), primary key (a, b) /*T![clustered_index] nonclustered */);",
},
{
"create table clustered_test(id int)",
"create table clustered_test(id int)",
},
{
"create database clustered_test",
"create database clustered_test",
},
{
"create database clustered",
"create database clustered",
},
{
"create table clustered (id int)",
"create table clustered (id int)",
},
}
for _, ca := range testCase {
re := binloginfo.AddSpecialComment(ca.input)
Expand Down
5 changes: 2 additions & 3 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,13 +381,12 @@ func (h *Handle) MergePartitionStats2GlobalStats(sc sessionctx.Context, opts map
}
errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, indexName)
}
err = types.ErrBuildGlobalLevelStatsFailed.GenWithStackByArgs(errMsg)
err = types.ErrPartitionStatsMissing.GenWithStackByArgs(errMsg)
return
}
statistics.CheckAnalyzeVerOnTable(partitionStats, &statsVer)
if statsVer != statistics.Version2 { // global-stats only support stats-ver2
return nil, fmt.Errorf("[stats]: global statistics for partitioned tables only available in statistics version2, please set tidb_analyze_version to 2")

return nil, fmt.Errorf("[stats]: some partition level statistics are not in statistics version 2, please set tidb_analyze_version to 2 and analyze the this table")
}
for i := 0; i < globalStats.Num; i++ {
ID := tableInfo.Columns[i].ID
Expand Down
Loading

0 comments on commit b8c5f03

Please sign in to comment.