Skip to content

Commit

Permalink
Merge branch 'release-5.0' into release-5.0-1ac53c546d9e
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka authored Apr 16, 2021
2 parents bce34f6 + 50c43ad commit 15e10c9
Show file tree
Hide file tree
Showing 38 changed files with 553 additions and 102 deletions.
9 changes: 9 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -821,6 +821,8 @@ func FormatSQL(sql string, pps variable.PreparedParams) stringutil.StringerFunc
var (
sessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal)
sessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral)
totalTiFlashQueryFailCounter = metrics.TiFlashQueryTotalCounter.WithLabelValues(metrics.LblError)
totalTiFlashQuerySuccCounter = metrics.TiFlashQueryTotalCounter.WithLabelValues(metrics.LblOK)
)

// FinishExecuteStmt is used to record some information after `ExecStmt` execution finished:
Expand Down Expand Up @@ -850,6 +852,13 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults boo
// `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`.
a.LogSlowQuery(txnTS, succ, hasMoreResults)
a.SummaryStmt(succ)
if sessVars.StmtCtx.IsTiFlash.Load() {
if succ {
totalTiFlashQuerySuccCounter.Inc()
} else {
totalTiFlashQueryFailCounter.Inc()
}
}
prevStmt := a.GetTextToLog()
if sessVars.EnableRedactLog {
sessVars.PrevStmt = FormatSQL(prevStmt, nil)
Expand Down
3 changes: 3 additions & 0 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -236,6 +236,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) {
}

func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) {
c.Skip("unstable")
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
defer func() {
Expand Down Expand Up @@ -509,6 +510,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) {
}

func (s *testSerialSuite2) TestFastAnalyze4GlobalStats(c *C) {
c.Skip("unstable")
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_enable_fast_analyze=1")
Expand Down Expand Up @@ -743,6 +745,7 @@ func (s *testSuite10) TestFailedAnalyzeRequest(c *C) {
}

func (s *testSuite1) TestExtractTopN(c *C) {
c.Skip("unstable")
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down
2 changes: 1 addition & 1 deletion executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,7 @@ func (getter *PessimisticLockCacheGetter) Get(_ context.Context, key kv.Key) ([]
}

func getPhysID(tblInfo *model.TableInfo, intVal int64) int64 {
pi := tblInfo.Partition
pi := tblInfo.GetPartitionInfo()
if pi == nil {
return tblInfo.ID
}
Expand Down
3 changes: 2 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2677,6 +2677,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E
return nil
}
if v.StoreType == kv.TiFlash {
sctx.IsTiFlash.Store(true)
partsExecutor := make([]Executor, 0, len(partitions))
for _, part := range partitions {
exec, err := buildNoRangeTableReader(b, v)
Expand Down Expand Up @@ -3012,7 +3013,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn
e.tableRequest.CollectRangeCounts = &collectTable
collectIndex := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges))
// Do not collect the feedback when the table is the partition table.
if collectIndex && tbl.Meta().Partition != nil {
if collectIndex && tbl.Meta().GetPartitionInfo() != nil {
collectIndex = false
}
if !collectIndex {
Expand Down
21 changes: 21 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4436,6 +4436,11 @@ func (s *testSuite6) TestUpdateJoin(c *C) {
tk.MustExec("insert into t7 values (5, 1, 'a')")
tk.MustExec("update t6, t7 set t6.v = t7.v where t6.id = t7.id and t7.x = 5")
tk.MustQuery("select v from t6").Check(testkit.Rows("a"))

tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1(id int primary key, v int, gv int GENERATED ALWAYS AS (v * 2) STORED)")
tk.MustExec("create table t2(id int, v int)")
tk.MustExec("update t1 tt1 inner join (select count(t1.id) a, t1.id from t1 left join t2 on t1.id = t2.id group by t1.id) x on tt1.id = x.id set tt1.v = tt1.v + x.a")
}

func (s *testSuite3) TestMaxOneRow(c *C) {
Expand Down Expand Up @@ -5539,6 +5544,22 @@ func (s *testSuiteP2) TestUnsignedFeedback(c *C) {
c.Assert(result.Rows()[2][6], Equals, "range:[0,+inf], keep order:false")
}

func (s *testSuiteP2) TestIssue23567(c *C) {
tk := testkit.NewTestKit(c, s.store)
oriProbability := statistics.FeedbackProbability.Load()
statistics.FeedbackProbability.Store(1.0)
defer func() { statistics.FeedbackProbability.Store(oriProbability) }()
failpoint.Enable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect", `return("")`)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))")
tk.MustExec("insert into t values (1, 1), (2, 2)")
tk.MustExec("analyze table t")
// The SQL should not panic.
tk.MustQuery("select count(distinct b) from t")
failpoint.Disable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect")
}

func (s *testSuite) TestSummaryFailedUpdate(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
46 changes: 46 additions & 0 deletions executor/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@ package executor_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/testkit"
)
Expand Down Expand Up @@ -176,6 +178,50 @@ PRIMARY KEY (pk1,pk2)) partition by hash(pk2) partitions 4;`)
tk.MustQuery("select /*+ INL_MERGE_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1"))
}

func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t_info_null")
tk.MustExec(`CREATE TABLE t_info_null (
id bigint(20) unsigned NOT NULL AUTO_INCREMENT,
date date NOT NULL,
media varchar(32) NOT NULL DEFAULT '0',
app varchar(32) NOT NULL DEFAULT '',
xxx bigint(20) NOT NULL DEFAULT '0',
PRIMARY KEY (id, date),
UNIQUE KEY idx_media_id (media, date, app)
) PARTITION BY RANGE COLUMNS(date) (
PARTITION p201912 VALUES LESS THAN ("2020-01-01"),
PARTITION p202001 VALUES LESS THAN ("2020-02-01"),
PARTITION p202002 VALUES LESS THAN ("2020-03-01"),
PARTITION p202003 VALUES LESS THAN ("2020-04-01"),
PARTITION p202004 VALUES LESS THAN ("2020-05-01"),
PARTITION p202005 VALUES LESS THAN ("2020-06-01"),
PARTITION p202006 VALUES LESS THAN ("2020-07-01"),
PARTITION p202007 VALUES LESS THAN ("2020-08-01"),
PARTITION p202008 VALUES LESS THAN ("2020-09-01"),
PARTITION p202009 VALUES LESS THAN ("2020-10-01"),
PARTITION p202010 VALUES LESS THAN ("2020-11-01"),
PARTITION p202011 VALUES LESS THAN ("2020-12-01")
)`)
is := infoschema.GetInfoSchema(tk.Se)
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null"))
c.Assert(err, IsNil)

tbInfo := tbl.Meta()
// Mock for a case that the tableInfo.Partition is not nil, but tableInfo.Partition.Enable is false.
// That may happen when upgrading from a old version TiDB.
tbInfo.Partition.Enable = false
tbInfo.Partition.Num = 0

tk.MustExec("set @@tidb_partition_prune_mode = 'static'")
tk.MustQuery("explain select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows("Batch_Point_Get_5 2.00 root table:t_info_null, index:idx_media_id(media, date, app) keep order:false, desc:false"))
tk.MustQuery("explain select * from t_info_null").Check(testkit.Rows("TableReader_5 10000.00 root data:TableFullScan_4",
"└─TableFullScan_4 10000.00 cop[tikv] table:t_info_null keep order:false, stats:pseudo"))
// No panic.
tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows())
}

func (s *globalIndexSuite) TestGlobalIndexScan(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists p")
Expand Down
2 changes: 2 additions & 0 deletions executor/simple_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,6 +604,8 @@ func (s *testFlushSuite) TestFlushPrivilegesPanic(c *C) {
}

func (s *testSuite3) TestDropPartitionStats(c *C) {
c.Skip("unstable")
// Use the testSerialSuite to fix the unstable test
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec(`create table t (
Expand Down
2 changes: 1 addition & 1 deletion executor/table_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ func (e *TableReaderExecutor) Close() error {
return err
}

// buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResut returned by the callee
// buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee
// to fetch all results.
func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) {
var builder distsql.RequestBuilder
Expand Down
33 changes: 33 additions & 0 deletions executor/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,3 +448,36 @@ func (s *tiflashTestSuite) TestMppApply(c *C) {
// table range scan with correlated access conditions
tk.MustQuery("select /*+ agg_to_cop(), hash_agg()*/ count(*) from x1 where b > any (select x2.a from x2 where x1.a = x2.a);").Check(testkit.Rows("2"))
}

func (s *tiflashTestSuite) TestTiFlashVirtualColumn(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1,t2,t3")
tk.MustExec("create table t1 (a bit(4), b bit(4), c bit(4) generated always as (a) virtual)")
tk.MustExec("alter table t1 set tiflash replica 1")
tb := testGetTableByName(c, tk.Se, "test", "t1")
err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true)
c.Assert(err, IsNil)
tk.MustExec("insert into t1(a,b) values(b'01',b'01'),(b'10',b'10'),(b'11',b'11')")

tk.MustExec("create table t2 (a int, b int, c int generated always as (a) virtual)")
tk.MustExec("alter table t2 set tiflash replica 1")
tb = testGetTableByName(c, tk.Se, "test", "t2")
err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true)
c.Assert(err, IsNil)
tk.MustExec("insert into t2(a,b) values(1,1),(2,2),(3,3)")

tk.MustExec("create table t3 (a bit(4), b bit(4), c bit(4) generated always as (b'01'+b'10') virtual)")
tk.MustExec("alter table t3 set tiflash replica 1")
tb = testGetTableByName(c, tk.Se, "test", "t3")
err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true)
c.Assert(err, IsNil)
tk.MustExec("insert into t3(a,b) values(b'01',b'01'),(b'10',b'10'),(b'11',b'11')")

tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"")
tk.MustExec("set @@session.tidb_allow_mpp=ON")

tk.MustQuery("select /*+ hash_agg() */ count(*) from t1 where c > b'01'").Check(testkit.Rows("2"))
tk.MustQuery("select /*+ hash_agg() */ count(*) from t2 where c > 1").Check(testkit.Rows("2"))
tk.MustQuery("select /*+ hash_agg() */ count(*) from t3 where c > b'01'").Check(testkit.Rows("3"))
}
2 changes: 1 addition & 1 deletion executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old
if sc.DupKeyAsWarning {
// For `UPDATE IGNORE`/`INSERT IGNORE ON DUPLICATE KEY UPDATE`
// If the new handle or unique index exists, this will avoid to remove the record.
err = tables.CheckHandleOrUniqueKeyExistForUpdateIgnoreOrInsertOnDupIgnore(ctx, sctx, t, newHandle, newData)
err = tables.CheckHandleOrUniqueKeyExistForUpdateIgnoreOrInsertOnDupIgnore(ctx, sctx, t, newHandle, newData, modified)
if err != nil {
if terr, ok := errors.Cause(err).(*terror.Error); sctx.GetSessionVars().StmtCtx.IgnoreNoPartition && ok && terr.Code() == errno.ErrNoPartitionForGivenValue {
return false, nil
Expand Down
8 changes: 8 additions & 0 deletions executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -798,6 +798,14 @@ func (s *testSuite4) TestInsertIgnoreOnDup(c *C) {
tk.MustExec("update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'")
tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '2' for key 'ukk1'"))
tk.MustQuery("select * from t5").Check(testkit.Rows("1 1 1 100", "1 3 2 200"))

tk.MustExec("drop table if exists t6")
tk.MustExec("create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b))")
tk.MustExec("insert into t6 select 10, 10, 20")
tk.MustExec("insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100")
tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 20"))
tk.MustExec("insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000")
tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 1000"))
}

func (s *testSuite4) TestInsertSetWithDefault(c *C) {
Expand Down
22 changes: 11 additions & 11 deletions expression/builtin_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,11 @@ func SetBinFlagOrBinStr(argTp *types.FieldType, resTp *types.FieldType) {
}
}

// addBinFlag add the binary flag to `tp` if its charset is binary
func addBinFlag(tp *types.FieldType) {
SetBinFlagOrBinStr(tp, tp)
}

type lengthFunctionClass struct {
baseFunctionClass
}
Expand Down Expand Up @@ -275,10 +280,10 @@ func (c *concatFunctionClass) getFunction(ctx sessionctx.Context, args []Express
if err != nil {
return nil, err
}
addBinFlag(bf.tp)
bf.tp.Flen = 0
for i := range args {
argType := args[i].GetType()
SetBinFlagOrBinStr(argType, bf.tp)

if argType.Flen < 0 {
bf.tp.Flen = mysql.MaxBlobWidth
Expand Down Expand Up @@ -350,9 +355,9 @@ func (c *concatWSFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
}
bf.tp.Flen = 0

addBinFlag(bf.tp)
for i := range args {
argType := args[i].GetType()
SetBinFlagOrBinStr(argType, bf.tp)

// skip separator param
if i != 0 {
Expand Down Expand Up @@ -2000,8 +2005,7 @@ func (c *lpadFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio
return nil, err
}
bf.tp.Flen = getFlen4LpadAndRpad(bf.ctx, args[1])
SetBinFlagOrBinStr(args[0].GetType(), bf.tp)
SetBinFlagOrBinStr(args[2].GetType(), bf.tp)
addBinFlag(bf.tp)

valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket)
maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64)
Expand Down Expand Up @@ -2133,8 +2137,7 @@ func (c *rpadFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio
return nil, err
}
bf.tp.Flen = getFlen4LpadAndRpad(bf.ctx, args[1])
SetBinFlagOrBinStr(args[0].GetType(), bf.tp)
SetBinFlagOrBinStr(args[2].GetType(), bf.tp)
addBinFlag(bf.tp)

valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket)
maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64)
Expand Down Expand Up @@ -2668,9 +2671,7 @@ func (c *makeSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expres
if err != nil {
return nil, err
}
for i, length := 0, len(args); i < length; i++ {
SetBinFlagOrBinStr(args[i].GetType(), bf.tp)
}
addBinFlag(bf.tp)
bf.tp.Flen = c.getFlen(bf.ctx, args)
if bf.tp.Flen > mysql.MaxBlobWidth {
bf.tp.Flen = mysql.MaxBlobWidth
Expand Down Expand Up @@ -3591,8 +3592,7 @@ func (c *insertFunctionClass) getFunction(ctx sessionctx.Context, args []Express
return nil, err
}
bf.tp.Flen = mysql.MaxBlobWidth
SetBinFlagOrBinStr(args[0].GetType(), bf.tp)
SetBinFlagOrBinStr(args[3].GetType(), bf.tp)
addBinFlag(bf.tp)

valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket)
maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64)
Expand Down
5 changes: 4 additions & 1 deletion expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase {
{"space(c_int_d)", mysql.TypeLongBlob, mysql.DefaultCharset, 0, mysql.MaxBlobWidth, types.UnspecifiedLength},
{"CONCAT(c_binary, c_int_d)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 40, types.UnspecifiedLength},
{"CONCAT(c_bchar, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 40, types.UnspecifiedLength},
{"CONCAT(c_bchar, 0x80)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 23, types.UnspecifiedLength},
{"CONCAT('T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0 | mysql.NotNullFlag, 4, types.UnspecifiedLength},
{"CONCAT('T', 'i', 'DB', c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 24, types.UnspecifiedLength},
{"CONCAT_WS('-', 'T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0 | mysql.NotNullFlag, 6, types.UnspecifiedLength},
Expand Down Expand Up @@ -451,8 +452,9 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase {
{"find_in_set(c_set , c_text_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 3, 0},
{"find_in_set(c_enum , c_text_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 3, 0},

{"make_set(c_int_d , c_text_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 65535, types.UnspecifiedLength},
{"make_set(c_int_d , c_text_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 65535, types.UnspecifiedLength},
{"make_set(c_bigint_d , c_text_d, c_binary)", mysql.TypeMediumBlob, charset.CharsetBin, mysql.BinaryFlag, 65556, types.UnspecifiedLength},
{"make_set(1 , c_text_d, 0x40)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 65535, types.UnspecifiedLength},

{"quote(c_int_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 42, types.UnspecifiedLength},
{"quote(c_bigint_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 42, types.UnspecifiedLength},
Expand All @@ -465,6 +467,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase {
{"convert(c_text_d using 'binary')", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength},

{"insert(c_varchar, c_int_d, c_int_d, c_varchar)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength},
{"insert(c_varchar, c_int_d, c_int_d, 0x40)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength},
{"insert(c_varchar, c_int_d, c_int_d, c_binary)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength},
{"insert(c_binary, c_int_d, c_int_d, c_varchar)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength},
{"insert(c_binary, c_int_d, c_int_d, c_binary)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength},
Expand Down
1 change: 1 addition & 0 deletions metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@ func RegisterMetrics() {
prometheus.MustRegister(ServerInfo)
prometheus.MustRegister(TokenGauge)
prometheus.MustRegister(ConfigStatus)
prometheus.MustRegister(TiFlashQueryTotalCounter)
prometheus.MustRegister(SmallTxnWriteDuration)
prometheus.MustRegister(TxnWriteThroughput)

Expand Down
8 changes: 8 additions & 0 deletions metrics/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,6 +220,14 @@ var (
Name: "status",
Help: "Status of the TiDB server configurations.",
}, []string{LblType})

TiFlashQueryTotalCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "tidb",
Subsystem: "server",
Name: "tiflash_query_total",
Help: "Counter of TiFlash queries.",
}, []string{LblResult})
)

// ExecuteErrorToLabel converts an execute error to label.
Expand Down
2 changes: 1 addition & 1 deletion planner/core/cacheable_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ func (checker *cacheableChecker) isPartitionTable(tn *ast.TableName) bool {
logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err))
return false
}
if tb.Meta().Partition != nil {
if tb.Meta().GetPartitionInfo() != nil {
return true
}
return false
Expand Down
Loading

0 comments on commit 15e10c9

Please sign in to comment.