Skip to content

Commit

Permalink
telemetry: Add telemetry information about builtin functions usage (p…
Browse files Browse the repository at this point in the history
  • Loading branch information
leiysky authored Jul 21, 2021
1 parent dbe1f17 commit 126c9b4
Show file tree
Hide file tree
Showing 5 changed files with 104 additions and 19 deletions.
18 changes: 13 additions & 5 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/telemetry"
"github.com/pingcap/tidb/types"
driver "github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -1178,14 +1179,21 @@ func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok
}

// newFunction chooses which expression.NewFunctionImpl() will be used.
func (er *expressionRewriter) newFunction(funcName string, retType *types.FieldType, args ...expression.Expression) (expression.Expression, error) {
func (er *expressionRewriter) newFunction(funcName string, retType *types.FieldType, args ...expression.Expression) (ret expression.Expression, err error) {
if er.disableFoldCounter > 0 {
return expression.NewFunctionBase(er.sctx, funcName, retType, args...)
ret, err = expression.NewFunctionBase(er.sctx, funcName, retType, args...)
} else if er.tryFoldCounter > 0 {
ret, err = expression.NewFunctionTryFold(er.sctx, funcName, retType, args...)
} else {
ret, err = expression.NewFunction(er.sctx, funcName, retType, args...)
}
if er.tryFoldCounter > 0 {
return expression.NewFunctionTryFold(er.sctx, funcName, retType, args...)
if err != nil {
return
}
return expression.NewFunction(er.sctx, funcName, retType, args...)
if scalarFunc, ok := ret.(*expression.ScalarFunction); ok {
telemetry.BuiltinFunctionsUsage(er.b.ctx.GetBuiltinFunctionUsage()).Inc(scalarFunc.Function.PbCode().String())
}
return
}

func (er *expressionRewriter) checkTimePrecision(ft *types.FieldType) error {
Expand Down
27 changes: 18 additions & 9 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,8 @@ type session struct {
idxUsageCollector *handle.SessionIndexUsageCollector

cache [1]ast.StmtNode

builtinFunctionUsage telemetry.BuiltinFunctionsUsage
}

var parserPool = &sync.Pool{New: func() interface{} { return parser.New() }}
Expand Down Expand Up @@ -2244,6 +2246,7 @@ func (s *session) Close() {
if s.idxUsageCollector != nil {
s.idxUsageCollector.Delete()
}
telemetry.GlobalBuiltinFunctionsUsage.Collect(s.GetBuiltinFunctionUsage())
bindValue := s.Value(bindinfo.SessionBindInfoKeyType)
if bindValue != nil {
bindValue.(*bindinfo.SessionHandle).Close()
Expand Down Expand Up @@ -2659,11 +2662,12 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) {
return nil, err
}
s := &session{
store: store,
sessionVars: variable.NewSessionVars(),
ddlOwnerChecker: dom.DDL().OwnerManager(),
client: store.GetClient(),
mppClient: store.GetMPPClient(),
store: store,
sessionVars: variable.NewSessionVars(),
ddlOwnerChecker: dom.DDL().OwnerManager(),
client: store.GetClient(),
mppClient: store.GetMPPClient(),
builtinFunctionUsage: make(telemetry.BuiltinFunctionsUsage),
}
if plannercore.PreparedPlanCacheEnabled() {
if opt != nil && opt.PreparedPlanCache != nil {
Expand Down Expand Up @@ -2692,10 +2696,11 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) {
// a lock context, which cause we can't call createSession directly.
func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, error) {
s := &session{
store: store,
sessionVars: variable.NewSessionVars(),
client: store.GetClient(),
mppClient: store.GetMPPClient(),
store: store,
sessionVars: variable.NewSessionVars(),
client: store.GetClient(),
mppClient: store.GetMPPClient(),
builtinFunctionUsage: make(telemetry.BuiltinFunctionsUsage),
}
if plannercore.PreparedPlanCacheEnabled() {
s.preparedPlanCache = kvcache.NewSimpleLRUCache(plannercore.PreparedPlanCacheCapacity,
Expand Down Expand Up @@ -3126,3 +3131,7 @@ func (s *session) updateTelemetryMetric(es *executor.ExecStmt) {
telemetryCTEUsage.WithLabelValues("notCTE").Inc()
}
}

func (s *session) GetBuiltinFunctionUsage() map[string]uint32 {
return s.builtinFunctionUsage
}
3 changes: 3 additions & 0 deletions sessionctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,9 @@ type Context interface {
StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64)
// GetTxnWriteThroughputSLI returns the TxnWriteThroughputSLI.
GetTxnWriteThroughputSLI() *sli.TxnWriteThroughputSLI
// GetBuiltinFunctionUsage returns the BuiltinFunctionUsage of current Context, which is not thread safe.
// Use primitive map type to prevent circular import. Should convert it to telemetry.BuiltinFunctionUsage before using.
GetBuiltinFunctionUsage() map[string]uint32
}

type basicCtxType int
Expand Down
70 changes: 65 additions & 5 deletions telemetry/data_window.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,11 +61,12 @@ const (
)

type windowData struct {
BeginAt time.Time `json:"beginAt"`
ExecuteCount uint64 `json:"executeCount"`
TiFlashUsage tiFlashUsageData `json:"tiFlashUsage"`
CoprCacheUsage coprCacheUsageData `json:"coprCacheUsage"`
SQLUsage sqlUsageData `json:"SQLUsage"`
BeginAt time.Time `json:"beginAt"`
ExecuteCount uint64 `json:"executeCount"`
TiFlashUsage tiFlashUsageData `json:"tiFlashUsage"`
CoprCacheUsage coprCacheUsageData `json:"coprCacheUsage"`
SQLUsage sqlUsageData `json:"SQLUsage"`
BuiltinFunctionsUsage map[string]uint32 `json:"builtinFunctionsUsage"`
}

type sqlType map[string]uint64
Expand All @@ -90,6 +91,60 @@ type tiFlashUsageData struct {
ExchangePushDown uint64 `json:"exchangePushDown"`
}

// builtinFunctionsUsageCollector collects builtin functions usage information and dump it into windowData.
type builtinFunctionsUsageCollector struct {
sync.Mutex

// Should acquire lock to access this
usageData BuiltinFunctionsUsage
}

// Merge BuiltinFunctionsUsage data
func (b *builtinFunctionsUsageCollector) Collect(usageData BuiltinFunctionsUsage) {
// TODO(leiysky): use multi-worker to collect the usage information so we can make this asynchronous
b.Lock()
defer b.Unlock()
b.usageData.Merge(usageData)
}

// Dump BuiltinFunctionsUsage data
func (b *builtinFunctionsUsageCollector) Dump() map[string]uint32 {
b.Lock()
ret := b.usageData
b.usageData = make(map[string]uint32)
b.Unlock()

return ret
}

// BuiltinFunctionsUsage is a map from ScalarFuncSig_name(string) to usage count(uint32)
type BuiltinFunctionsUsage map[string]uint32

// Inc will increase the usage count of scalar function by 1
func (b BuiltinFunctionsUsage) Inc(scalarFuncSigName string) {
v, ok := b[scalarFuncSigName]
if !ok {
b[scalarFuncSigName] = 1
} else {
b[scalarFuncSigName] = v + 1
}
}

// Merge BuiltinFunctionsUsage data
func (b BuiltinFunctionsUsage) Merge(usageData BuiltinFunctionsUsage) {
for k, v := range usageData {
prev, ok := b[k]
if !ok {
b[k] = v
} else {
b[k] = prev + v
}
}
}

// GlobalBuiltinFunctionsUsage is used to collect builtin functions usage information
var GlobalBuiltinFunctionsUsage = &builtinFunctionsUsageCollector{usageData: make(BuiltinFunctionsUsage)}

var (
rotatedSubWindows []*windowData
subWindowsLock = sync.RWMutex{}
Expand Down Expand Up @@ -186,6 +241,7 @@ func RotateSubWindow() {
SQLTotal: 0,
SQLType: make(sqlType),
},
BuiltinFunctionsUsage: GlobalBuiltinFunctionsUsage.Dump(),
}

err := readSQLMetric(time.Now(), &thisSubWindow.SQLUsage)
Expand Down Expand Up @@ -243,6 +299,10 @@ func getWindowData() []*windowData {
thisWindow.CoprCacheUsage.GTE100 += rotatedSubWindows[i].CoprCacheUsage.GTE100
thisWindow.SQLUsage.SQLTotal = rotatedSubWindows[i].SQLUsage.SQLTotal - startWindow.SQLUsage.SQLTotal
thisWindow.SQLUsage.SQLType = calDeltaSQLTypeMap(rotatedSubWindows[i].SQLUsage.SQLType, startWindow.SQLUsage.SQLType)

mergedBuiltinFunctionsUsage := BuiltinFunctionsUsage(thisWindow.BuiltinFunctionsUsage)
mergedBuiltinFunctionsUsage.Merge(BuiltinFunctionsUsage(rotatedSubWindows[i].BuiltinFunctionsUsage))
thisWindow.BuiltinFunctionsUsage = mergedBuiltinFunctionsUsage
aggregatedSubWindows++
i++
}
Expand Down
5 changes: 5 additions & 0 deletions util/mock/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,6 +158,11 @@ func (c *Context) GetInfoSchema() sessionctx.InfoschemaMetaVersion {
return nil
}

// GetBuiltinFunctionUsage implements sessionctx.Context GetBuiltinFunctionUsage interface.
func (c *Context) GetBuiltinFunctionUsage() map[string]uint32 {
return make(map[string]uint32)
}

// GetGlobalSysVar implements GlobalVarAccessor GetGlobalSysVar interface.
func (c *Context) GetGlobalSysVar(ctx sessionctx.Context, name string) (string, error) {
v := variable.GetSysVar(name)
Expand Down

0 comments on commit 126c9b4

Please sign in to comment.