90 Star 491 Fork 151

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

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
functions.go 11.84 KB
一键复制 编辑 原始数据 按行查看 历史
// Copyright 2015 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 ast
import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/util/types"
)
var (
_ FuncNode = &AggregateFuncExpr{}
_ FuncNode = &FuncCallExpr{}
_ FuncNode = &FuncCastExpr{}
)
// List scalar function names.
const (
LogicAnd = "and"
Cast = "cast"
LeftShift = "leftshift"
RightShift = "rightshift"
LogicOr = "or"
GE = "ge"
LE = "le"
EQ = "eq"
NE = "ne"
LT = "lt"
GT = "gt"
Plus = "plus"
Minus = "minus"
And = "bitand"
Or = "bitor"
Mod = "mod"
Xor = "bitxor"
Div = "div"
Mul = "mul"
UnaryNot = "not" // Avoid name conflict with Not in github/pingcap/check.
BitNeg = "bitneg"
IntDiv = "intdiv"
LogicXor = "xor"
NullEQ = "nulleq"
UnaryPlus = "unaryplus"
UnaryMinus = "unaryminus"
In = "in"
Like = "like"
Case = "case"
Regexp = "regexp"
IsNull = "isnull"
IsTruth = "istrue" // Avoid name conflict with IsTrue in github/pingcap/check.
IsFalsity = "isfalse" // Avoid name conflict with IsFalse in github/pingcap/check.
RowFunc = "row"
SetVar = "setvar"
GetVar = "getvar"
Values = "values"
BitCount = "bit_count"
// common functions
Coalesce = "coalesce"
Greatest = "greatest"
Least = "least"
Interval = "interval"
// math functions
Abs = "abs"
Acos = "acos"
Asin = "asin"
Atan = "atan"
Atan2 = "atan2"
Ceil = "ceil"
Ceiling = "ceiling"
Conv = "conv"
Cos = "cos"
Cot = "cot"
CRC32 = "crc32"
Degrees = "degrees"
Exp = "exp"
Floor = "floor"
Ln = "ln"
Log = "log"
Log2 = "log2"
Log10 = "log10"
PI = "pi"
Pow = "pow"
Power = "power"
Radians = "radians"
Rand = "rand"
Round = "round"
Sign = "sign"
Sin = "sin"
Sqrt = "sqrt"
Tan = "tan"
Truncate = "truncate"
// time functions
AddDate = "adddate"
AddTime = "addtime"
ConvertTz = "convert_tz"
Curdate = "curdate"
CurrentDate = "current_date"
CurrentTime = "current_time"
CurrentTimestamp = "current_timestamp"
Curtime = "curtime"
Date = "date"
DateLiteral = "dateliteral"
DateAdd = "date_add"
DateFormat = "date_format"
DateSub = "date_sub"
DateDiff = "datediff"
Day = "day"
DayName = "dayname"
DayOfMonth = "dayofmonth"
DayOfWeek = "dayofweek"
DayOfYear = "dayofyear"
Extract = "extract"
FromDays = "from_days"
FromUnixTime = "from_unixtime"
GetFormat = "get_format"
Hour = "hour"
LocalTime = "localtime"
LocalTimestamp = "localtimestamp"
MakeDate = "makedate"
MakeTime = "maketime"
MicroSecond = "microsecond"
Minute = "minute"
Month = "month"
MonthName = "monthname"
Now = "now"
PeriodAdd = "period_add"
PeriodDiff = "period_diff"
Quarter = "quarter"
SecToTime = "sec_to_time"
Second = "second"
StrToDate = "str_to_date"
SubDate = "subdate"
SubTime = "subtime"
Sysdate = "sysdate"
Time = "time"
TimeLiteral = "timeliteral"
TimeFormat = "time_format"
TimeToSec = "time_to_sec"
TimeDiff = "timediff"
Timestamp = "timestamp"
TimestampLiteral = "timestampliteral"
TimestampAdd = "timestampadd"
TimestampDiff = "timestampdiff"
ToDays = "to_days"
ToSeconds = "to_seconds"
UnixTimestamp = "unix_timestamp"
UTCDate = "utc_date"
UTCTime = "utc_time"
UTCTimestamp = "utc_timestamp"
Week = "week"
Weekday = "weekday"
WeekOfYear = "weekofyear"
Year = "year"
YearWeek = "yearweek"
LastDay = "last_day"
// string functions
ASCII = "ascii"
Bin = "bin"
Concat = "concat"
ConcatWS = "concat_ws"
Convert = "convert"
Elt = "elt"
ExportSet = "export_set"
Field = "field"
Format = "format"
FromBase64 = "from_base64"
InsertFunc = "insert_func"
Instr = "instr"
Lcase = "lcase"
Left = "left"
Length = "length"
LoadFile = "load_file"
Locate = "locate"
Lower = "lower"
Lpad = "lpad"
LTrim = "ltrim"
MakeSet = "make_set"
Mid = "mid"
Oct = "oct"
Ord = "ord"
Position = "position"
Quote = "quote"
Repeat = "repeat"
Replace = "replace"
Reverse = "reverse"
Right = "right"
RTrim = "rtrim"
Space = "space"
Strcmp = "strcmp"
Substring = "substring"
Substr = "substr"
SubstringIndex = "substring_index"
ToBase64 = "to_base64"
Trim = "trim"
Upper = "upper"
Ucase = "ucase"
Hex = "hex"
Unhex = "unhex"
Rpad = "rpad"
BitLength = "bit_length"
CharFunc = "char_func"
CharLength = "char_length"
CharacterLength = "character_length"
FindInSet = "find_in_set"
// information functions
Benchmark = "benchmark"
Charset = "charset"
Coercibility = "coercibility"
Collation = "collation"
ConnectionID = "connection_id"
CurrentUser = "current_user"
Database = "database"
FoundRows = "found_rows"
LastInsertId = "last_insert_id"
RowCount = "row_count"
Schema = "schema"
SessionUser = "session_user"
SystemUser = "system_user"
User = "user"
Version = "version"
TiDBVersion = "tidb_version"
// control functions
If = "if"
Ifnull = "ifnull"
Nullif = "nullif"
// miscellaneous functions
AnyValue = "any_value"
DefaultFunc = "default_func"
InetAton = "inet_aton"
InetNtoa = "inet_ntoa"
Inet6Aton = "inet6_aton"
Inet6Ntoa = "inet6_ntoa"
IsFreeLock = "is_free_lock"
IsIPv4 = "is_ipv4"
IsIPv4Compat = "is_ipv4_compat"
IsIPv4Mapped = "is_ipv4_mapped"
IsIPv6 = "is_ipv6"
IsUsedLock = "is_used_lock"
MasterPosWait = "master_pos_wait"
NameConst = "name_const"
ReleaseAllLocks = "release_all_locks"
Sleep = "sleep"
UUID = "uuid"
UUIDShort = "uuid_short"
// get_lock() and release_lock() is parsed but do nothing.
// It is used for preventing error in Ruby's activerecord migrations.
GetLock = "get_lock"
ReleaseLock = "release_lock"
// encryption and compression functions
AesDecrypt = "aes_decrypt"
AesEncrypt = "aes_encrypt"
Compress = "compress"
Decode = "decode"
DesDecrypt = "des_decrypt"
DesEncrypt = "des_encrypt"
Encode = "encode"
Encrypt = "encrypt"
MD5 = "md5"
OldPassword = "old_password"
PasswordFunc = "password_func"
RandomBytes = "random_bytes"
SHA1 = "sha1"
SHA = "sha"
SHA2 = "sha2"
Uncompress = "uncompress"
UncompressedLength = "uncompressed_length"
ValidatePasswordStrength = "validate_password_strength"
// json functions
JSONType = "json_type"
JSONExtract = "json_extract"
JSONUnquote = "json_unquote"
JSONArray = "json_array"
JSONObject = "json_object"
JSONMerge = "json_merge"
JSONValid = "json_valid"
JSONSet = "json_set"
JSONInsert = "json_insert"
JSONReplace = "json_replace"
JSONRemove = "json_remove"
JSONContains = "json_contains"
)
// FuncCallExpr is for function expression.
type FuncCallExpr struct {
funcNode
// FnName is the function name.
FnName model.CIStr
// Args is the function args.
Args []ExprNode
}
// Accept implements Node interface.
func (n *FuncCallExpr) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*FuncCallExpr)
for i, val := range n.Args {
node, ok := val.Accept(v)
if !ok {
return n, false
}
n.Args[i] = node.(ExprNode)
}
return v.Leave(n)
}
// CastFunctionType is the type for cast function.
type CastFunctionType int
// CastFunction types
const (
CastFunction CastFunctionType = iota + 1
CastConvertFunction
CastBinaryOperator
)
// FuncCastExpr is the cast function converting value to another type, e.g, cast(expr AS signed).
// See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html
type FuncCastExpr struct {
funcNode
// Expr is the expression to be converted.
Expr ExprNode
// Tp is the conversion type.
Tp *types.FieldType
// FunctionType is either Cast, Convert or Binary.
FunctionType CastFunctionType
}
// Accept implements Node Accept interface.
func (n *FuncCastExpr) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*FuncCastExpr)
node, ok := n.Expr.Accept(v)
if !ok {
return n, false
}
n.Expr = node.(ExprNode)
return v.Leave(n)
}
// TrimDirectionType is the type for trim direction.
type TrimDirectionType int
const (
// TrimBothDefault trims from both direction by default.
TrimBothDefault TrimDirectionType = iota
// TrimBoth trims from both direction with explicit notation.
TrimBoth
// TrimLeading trims from left.
TrimLeading
// TrimTrailing trims from right.
TrimTrailing
)
// DateArithType is type for DateArith type.
type DateArithType byte
const (
// DateArithAdd is to run adddate or date_add function option.
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-add
DateArithAdd DateArithType = iota + 1
// DateArithSub is to run subdate or date_sub function option.
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-sub
DateArithSub
)
const (
// AggFuncCount is the name of Count function.
AggFuncCount = "count"
// AggFuncSum is the name of Sum function.
AggFuncSum = "sum"
// AggFuncAvg is the name of Avg function.
AggFuncAvg = "avg"
// AggFuncFirstRow is the name of FirstRowColumn function.
AggFuncFirstRow = "firstrow"
// AggFuncMax is the name of max function.
AggFuncMax = "max"
// AggFuncMin is the name of min function.
AggFuncMin = "min"
// AggFuncGroupConcat is the name of group_concat function.
AggFuncGroupConcat = "group_concat"
)
// AggregateFuncExpr represents aggregate function expression.
type AggregateFuncExpr struct {
funcNode
// F is the function name.
F string
// Args is the function args.
Args []ExprNode
// Distinct is true, function hence only aggregate distinct values.
// For example, column c1 values are "1", "2", "2", "sum(c1)" is "5",
// but "sum(distinct c1)" is "3".
Distinct bool
}
// Accept implements Node Accept interface.
func (n *AggregateFuncExpr) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*AggregateFuncExpr)
for i, val := range n.Args {
node, ok := val.Accept(v)
if !ok {
return n, false
}
n.Args[i] = node.(ExprNode)
}
return v.Leave(n)
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v1.0.8

搜索帮助

0d507c66 1850385 C8b1a773 1850385