90 Star 491 Fork 151

平凯星辰(北京)科技有限公司/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
cacheable_checker.go 2.01 KB
一键复制 编辑 原始数据 按行查看 历史
// Copyright 2017 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package plan
import (
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
)
// Cacheable checks whether the input ast is cacheable.
func Cacheable(node ast.Node) bool {
if _, isSelect := node.(*ast.SelectStmt); !isSelect {
return false
}
checker := cacheableChecker{
cacheable: true,
}
node.Accept(&checker)
return checker.cacheable
}
// cacheableChecker checks whether a query's plan can be cached, querys that:
// 1. have ExistsSubqueryExpr, or
// 2. have VariableExpr
// will not be cached currently.
// NOTE: we can add more rules in the future.
type cacheableChecker struct {
cacheable bool
}
// Enter implements Visitor interface.
func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) {
switch node := in.(type) {
case *ast.VariableExpr, *ast.ExistsSubqueryExpr:
checker.cacheable = false
return in, true
case *ast.FuncCallExpr:
if _, found := expression.UnCacheableFunctions[node.FnName.L]; found {
checker.cacheable = false
return in, true
}
case *ast.Limit:
if node.Count != nil {
if _, isParamMarker := node.Count.(*ast.ParamMarkerExpr); isParamMarker {
checker.cacheable = false
return in, true
}
}
if node.Offset != nil {
if _, isParamMarker := node.Offset.(*ast.ParamMarkerExpr); isParamMarker {
checker.cacheable = false
return in, true
}
}
}
return in, false
}
// Leave implements Visitor interface.
func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) {
return in, checker.cacheable
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v1.1.0-alpha

搜索帮助

0d507c66 1850385 C8b1a773 1850385