forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
/
FunctionRegistry.scala
211 lines (186 loc) · 7.14 KB
/
FunctionRegistry.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst.analysis
import scala.reflect.ClassTag
import scala.util.{Failure, Success, Try}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.StringKeyHashMap
/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */
trait FunctionRegistry {
def registerFunction(name: String, builder: FunctionBuilder): Unit
@throws[AnalysisException]("If function does not exist")
def lookupFunction(name: String, children: Seq[Expression]): Expression
}
class SimpleFunctionRegistry extends FunctionRegistry {
private val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive = false)
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
functionBuilders.put(name, builder)
}
override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
val func = functionBuilders.get(name).getOrElse {
throw new AnalysisException(s"undefined function $name")
}
func(children)
}
}
/**
* A trivial catalog that returns an error when a function is requested. Used for testing when all
* functions are already filled in and the analyzer needs only to resolve attribute references.
*/
object EmptyFunctionRegistry extends FunctionRegistry {
override def registerFunction(name: String, builder: FunctionBuilder): Unit = {
throw new UnsupportedOperationException
}
override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
throw new UnsupportedOperationException
}
}
object FunctionRegistry {
type FunctionBuilder = Seq[Expression] => Expression
val expressions: Map[String, FunctionBuilder] = Map(
// misc non-aggregate functions
expression[Abs]("abs"),
expression[CreateArray]("array"),
expression[Coalesce]("coalesce"),
expression[Explode]("explode"),
expression[Greatest]("greatest"),
expression[If]("if"),
expression[IsNull]("isnull"),
expression[IsNotNull]("isnotnull"),
expression[Least]("least"),
expression[Coalesce]("nvl"),
expression[Rand]("rand"),
expression[Randn]("randn"),
expression[CreateStruct]("struct"),
expression[CreateNamedStruct]("named_struct"),
expression[Sqrt]("sqrt"),
// math functions
expression[Acos]("acos"),
expression[Asin]("asin"),
expression[Atan]("atan"),
expression[Atan2]("atan2"),
expression[Bin]("bin"),
expression[Cbrt]("cbrt"),
expression[Ceil]("ceil"),
expression[Ceil]("ceiling"),
expression[Cos]("cos"),
expression[EulerNumber]("e"),
expression[Exp]("exp"),
expression[Expm1]("expm1"),
expression[Floor]("floor"),
expression[Factorial]("factorial"),
expression[Hypot]("hypot"),
expression[Hex]("hex"),
expression[Logarithm]("log"),
expression[Log]("ln"),
expression[Log10]("log10"),
expression[Log1p]("log1p"),
expression[UnaryMinus]("negative"),
expression[Pi]("pi"),
expression[Log2]("log2"),
expression[Pow]("pow"),
expression[Pow]("power"),
expression[UnaryPositive]("positive"),
expression[Rint]("rint"),
expression[Round]("round"),
expression[ShiftLeft]("shiftleft"),
expression[ShiftRight]("shiftright"),
expression[ShiftRightUnsigned]("shiftrightunsigned"),
expression[Signum]("sign"),
expression[Signum]("signum"),
expression[Sin]("sin"),
expression[Sinh]("sinh"),
expression[Tan]("tan"),
expression[Tanh]("tanh"),
expression[ToDegrees]("degrees"),
expression[ToRadians]("radians"),
// misc functions
expression[Md5]("md5"),
expression[Sha2]("sha2"),
expression[Sha1]("sha1"),
expression[Sha1]("sha"),
expression[Crc32]("crc32"),
// aggregate functions
expression[Average]("avg"),
expression[Count]("count"),
expression[First]("first"),
expression[Last]("last"),
expression[Max]("max"),
expression[Min]("min"),
expression[Sum]("sum"),
// string functions
expression[Ascii]("ascii"),
expression[Base64]("base64"),
expression[Encode]("encode"),
expression[Decode]("decode"),
expression[StringInstr]("instr"),
expression[Lower]("lcase"),
expression[Lower]("lower"),
expression[StringLength]("length"),
expression[Levenshtein]("levenshtein"),
expression[StringLocate]("locate"),
expression[StringLPad]("lpad"),
expression[StringTrimLeft]("ltrim"),
expression[StringFormat]("printf"),
expression[StringRPad]("rpad"),
expression[StringRepeat]("repeat"),
expression[StringReverse]("reverse"),
expression[StringTrimRight]("rtrim"),
expression[StringSpace]("space"),
expression[StringSplit]("split"),
expression[Substring]("substr"),
expression[Substring]("substring"),
expression[StringTrim]("trim"),
expression[UnBase64]("unbase64"),
expression[Upper]("ucase"),
expression[Unhex]("unhex"),
expression[Upper]("upper"),
// datetime functions
expression[CurrentDate]("current_date"),
expression[CurrentTimestamp]("current_timestamp")
)
val builtin: FunctionRegistry = {
val fr = new SimpleFunctionRegistry
expressions.foreach { case (name, builder) => fr.registerFunction(name, builder) }
fr
}
/** See usage above. */
private def expression[T <: Expression](name: String)
(implicit tag: ClassTag[T]): (String, FunctionBuilder) = {
// See if we can find a constructor that accepts Seq[Expression]
val varargCtor = Try(tag.runtimeClass.getDeclaredConstructor(classOf[Seq[_]])).toOption
val builder = (expressions: Seq[Expression]) => {
if (varargCtor.isDefined) {
// If there is an apply method that accepts Seq[Expression], use that one.
varargCtor.get.newInstance(expressions).asInstanceOf[Expression]
} else {
// Otherwise, find an ctor method that matches the number of arguments, and use that.
val params = Seq.fill(expressions.size)(classOf[Expression])
val f = Try(tag.runtimeClass.getDeclaredConstructor(params : _*)) match {
case Success(e) =>
e
case Failure(e) =>
throw new AnalysisException(s"Invalid number of arguments for function $name")
}
f.newInstance(expressions : _*).asInstanceOf[Expression]
}
}
(name, builder)
}
}