@@ -54,46 +54,31 @@ private[hive] abstract class HiveFunctionRegistry
5454 val functionClassName = functionInfo.getFunctionClass.getName
5555
5656 if (classOf [UDF ].isAssignableFrom(functionInfo.getFunctionClass)) {
57- HiveSimpleUdf (functionClassName, children)
57+ HiveSimpleUdf (new HiveFunctionWrapper ( functionClassName) , children)
5858 } else if (classOf [GenericUDF ].isAssignableFrom(functionInfo.getFunctionClass)) {
59- HiveGenericUdf (functionClassName, children)
59+ HiveGenericUdf (new HiveFunctionWrapper ( functionClassName) , children)
6060 } else if (
6161 classOf [AbstractGenericUDAFResolver ].isAssignableFrom(functionInfo.getFunctionClass)) {
62- HiveGenericUdaf (functionClassName, children)
62+ HiveGenericUdaf (new HiveFunctionWrapper ( functionClassName) , children)
6363 } else if (classOf [UDAF ].isAssignableFrom(functionInfo.getFunctionClass)) {
64- HiveUdaf (functionClassName, children)
64+ HiveUdaf (new HiveFunctionWrapper ( functionClassName) , children)
6565 } else if (classOf [GenericUDTF ].isAssignableFrom(functionInfo.getFunctionClass)) {
66- HiveGenericUdtf (functionClassName, Nil , children)
66+ HiveGenericUdtf (new HiveFunctionWrapper ( functionClassName) , Nil , children)
6767 } else {
6868 sys.error(s " No handler for udf ${functionInfo.getFunctionClass}" )
6969 }
7070 }
7171}
7272
73- private [hive] trait HiveFunctionFactory {
74- val functionClassName : String
75-
76- def createFunction [UDFType ]() =
77- getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf [UDFType ]
78- }
79-
80- private [hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory {
81- self : Product =>
82-
83- type UDFType
73+ private [hive] case class HiveSimpleUdf (funcWrapper : HiveFunctionWrapper , children : Seq [Expression ])
74+ extends Expression with HiveInspectors with Logging {
8475 type EvaluatedType = Any
76+ type UDFType = UDF
8577
8678 def nullable = true
8779
88- lazy val function = createFunction[UDFType ]()
89-
90- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
91- }
92-
93- private [hive] case class HiveSimpleUdf (functionClassName : String , children : Seq [Expression ])
94- extends HiveUdf with HiveInspectors {
95-
96- type UDFType = UDF
80+ @ transient
81+ lazy val function = funcWrapper.createFunction[UDFType ]()
9782
9883 @ transient
9984 protected lazy val method =
@@ -131,6 +116,8 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[
131116 .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _* ): _* ),
132117 returnInspector)
133118 }
119+
120+ override def toString = s " $nodeName# ${funcWrapper.functionClassName}( ${children.mkString(" ," )}) "
134121}
135122
136123// Adapter from Catalyst ExpressionResult to Hive DeferredObject
@@ -144,16 +131,23 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector)
144131 override def get (): AnyRef = wrap(func(), oi)
145132}
146133
147- private [hive] case class HiveGenericUdf (functionClassName : String , children : Seq [Expression ])
148- extends HiveUdf with HiveInspectors {
134+ private [hive] case class HiveGenericUdf (funcWrapper : HiveFunctionWrapper , children : Seq [Expression ])
135+ extends Expression with HiveInspectors with Logging {
149136 type UDFType = GenericUDF
137+ type EvaluatedType = Any
138+
139+ def nullable = true
140+
141+ @ transient
142+ lazy val function = funcWrapper.createFunction[UDFType ]()
150143
151144 @ transient
152145 protected lazy val argumentInspectors = children.map(toInspector)
153146
154147 @ transient
155- protected lazy val returnInspector =
148+ protected lazy val returnInspector = {
156149 function.initializeAndFoldConstants(argumentInspectors.toArray)
150+ }
157151
158152 @ transient
159153 protected lazy val isUDFDeterministic = {
@@ -183,18 +177,19 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq
183177 }
184178 unwrap(function.evaluate(deferedObjects), returnInspector)
185179 }
180+
181+ override def toString = s " $nodeName# ${funcWrapper.functionClassName}( ${children.mkString(" ," )}) "
186182}
187183
188184private [hive] case class HiveGenericUdaf (
189- functionClassName : String ,
185+ funcWrapper : HiveFunctionWrapper ,
190186 children : Seq [Expression ]) extends AggregateExpression
191- with HiveInspectors
192- with HiveFunctionFactory {
187+ with HiveInspectors {
193188
194189 type UDFType = AbstractGenericUDAFResolver
195190
196191 @ transient
197- protected lazy val resolver : AbstractGenericUDAFResolver = createFunction()
192+ protected lazy val resolver : AbstractGenericUDAFResolver = funcWrapper. createFunction()
198193
199194 @ transient
200195 protected lazy val objectInspector = {
@@ -209,22 +204,22 @@ private[hive] case class HiveGenericUdaf(
209204
210205 def nullable : Boolean = true
211206
212- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
207+ override def toString = s " $nodeName# ${funcWrapper. functionClassName} ( ${children.mkString(" ," )}) "
213208
214- def newInstance () = new HiveUdafFunction (functionClassName , children, this )
209+ def newInstance () = new HiveUdafFunction (funcWrapper , children, this )
215210}
216211
217212/** It is used as a wrapper for the hive functions which uses UDAF interface */
218213private [hive] case class HiveUdaf (
219- functionClassName : String ,
214+ funcWrapper : HiveFunctionWrapper ,
220215 children : Seq [Expression ]) extends AggregateExpression
221- with HiveInspectors
222- with HiveFunctionFactory {
216+ with HiveInspectors {
223217
224218 type UDFType = UDAF
225219
226220 @ transient
227- protected lazy val resolver : AbstractGenericUDAFResolver = new GenericUDAFBridge (createFunction())
221+ protected lazy val resolver : AbstractGenericUDAFResolver =
222+ new GenericUDAFBridge (funcWrapper.createFunction())
228223
229224 @ transient
230225 protected lazy val objectInspector = {
@@ -239,10 +234,10 @@ private[hive] case class HiveUdaf(
239234
240235 def nullable : Boolean = true
241236
242- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
237+ override def toString = s " $nodeName# ${funcWrapper. functionClassName} ( ${children.mkString(" ," )}) "
243238
244239 def newInstance () =
245- new HiveUdafFunction (functionClassName , children, this , true )
240+ new HiveUdafFunction (funcWrapper , children, this , true )
246241}
247242
248243/**
@@ -257,13 +252,13 @@ private[hive] case class HiveUdaf(
257252 * user defined aggregations, which have clean semantics even in a partitioned execution.
258253 */
259254private [hive] case class HiveGenericUdtf (
260- functionClassName : String ,
255+ funcWrapper : HiveFunctionWrapper ,
261256 aliasNames : Seq [String ],
262257 children : Seq [Expression ])
263- extends Generator with HiveInspectors with HiveFunctionFactory {
258+ extends Generator with HiveInspectors {
264259
265260 @ transient
266- protected lazy val function : GenericUDTF = createFunction()
261+ protected lazy val function : GenericUDTF = funcWrapper. createFunction()
267262
268263 @ transient
269264 protected lazy val inputInspectors = children.map(_.dataType).map(toInspector)
@@ -320,25 +315,24 @@ private[hive] case class HiveGenericUdtf(
320315 }
321316 }
322317
323- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
318+ override def toString = s " $nodeName# ${funcWrapper. functionClassName} ( ${children.mkString(" ," )}) "
324319}
325320
326321private [hive] case class HiveUdafFunction (
327- functionClassName : String ,
322+ funcWrapper : HiveFunctionWrapper ,
328323 exprs : Seq [Expression ],
329324 base : AggregateExpression ,
330325 isUDAFBridgeRequired : Boolean = false )
331326 extends AggregateFunction
332- with HiveInspectors
333- with HiveFunctionFactory {
327+ with HiveInspectors {
334328
335329 def this () = this (null , null , null )
336330
337331 private val resolver =
338332 if (isUDAFBridgeRequired) {
339- new GenericUDAFBridge (createFunction[UDAF ]())
333+ new GenericUDAFBridge (funcWrapper. createFunction[UDAF ]())
340334 } else {
341- createFunction[AbstractGenericUDAFResolver ]()
335+ funcWrapper. createFunction[AbstractGenericUDAFResolver ]()
342336 }
343337
344338 private val inspectors = exprs.map(_.dataType).map(toInspector).toArray
@@ -361,3 +355,4 @@ private[hive] case class HiveUdafFunction(
361355 function.iterate(buffer, inputs)
362356 }
363357}
358+
0 commit comments