@@ -54,47 +54,80 @@ private[hive] abstract class HiveFunctionRegistry
54
54
val functionClassName = functionInfo.getFunctionClass.getName
55
55
56
56
if (classOf [UDF ].isAssignableFrom(functionInfo.getFunctionClass)) {
57
- HiveSimpleUdf (functionClassName, children)
57
+ HiveSimpleUdf (new HiveFunctionCache ( functionClassName) , children)
58
58
} else if (classOf [GenericUDF ].isAssignableFrom(functionInfo.getFunctionClass)) {
59
- HiveGenericUdf (functionClassName, children)
59
+ HiveGenericUdf (new HiveFunctionCache ( functionClassName) , children)
60
60
} else if (
61
61
classOf [AbstractGenericUDAFResolver ].isAssignableFrom(functionInfo.getFunctionClass)) {
62
- HiveGenericUdaf (functionClassName, children)
62
+ HiveGenericUdaf (new HiveFunctionCache ( functionClassName) , children)
63
63
} else if (classOf [UDAF ].isAssignableFrom(functionInfo.getFunctionClass)) {
64
- HiveUdaf (functionClassName, children)
64
+ HiveUdaf (new HiveFunctionCache ( functionClassName) , children)
65
65
} else if (classOf [GenericUDTF ].isAssignableFrom(functionInfo.getFunctionClass)) {
66
- HiveGenericUdtf (functionClassName, Nil , children)
66
+ HiveGenericUdtf (new HiveFunctionCache ( functionClassName) , Nil , children)
67
67
} else {
68
68
sys.error(s " No handler for udf ${functionInfo.getFunctionClass}" )
69
69
}
70
70
}
71
71
}
72
72
73
- private [hive] trait HiveFunctionFactory {
74
- val functionClassName : String
73
+ /**
74
+ * This class provides the UDF creation and also the UDF instance serialization and
75
+ * de-serialization cross process boundary.
76
+ * @param functionClassName UDF class name
77
+ */
78
+ class HiveFunctionCache (var functionClassName : String ) extends java.io.Externalizable {
79
+ // for Seriliazation
80
+ def this () = this (null )
75
81
76
- def createFunction [UDFType ]() =
77
- getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf [UDFType ]
78
- }
82
+ private var instance : Any = null
79
83
80
- private [hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory {
81
- self : Product =>
84
+ def writeExternal (out : java.io.ObjectOutput ) {
85
+ // Some of the UDF are serializable, but some not
86
+ // Hive Utilities can handle both case
87
+ val baos = new java.io.ByteArrayOutputStream ()
88
+ HiveShim .serializePlan(instance, baos)
89
+ val functionInBytes = baos.toByteArray
82
90
83
- type UDFType
84
- type EvaluatedType = Any
91
+ // output the function name
92
+ out.writeUTF(functionClassName)
85
93
86
- def nullable = true
94
+ // output the function bytes
95
+ out.writeInt(functionInBytes.length)
96
+ out.write(functionInBytes, 0 , functionInBytes.length)
97
+ }
87
98
88
- lazy val function = createFunction[UDFType ]()
99
+ def readExternal (in : java.io.ObjectInput ) {
100
+ // read the function name
101
+ functionClassName = in.readUTF()
89
102
90
- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
91
- }
103
+ // read the function in bytes
104
+ val functionInBytesLength = in.readInt()
105
+ val functionInBytes = new Array [Byte ](functionInBytesLength)
106
+ in.read(functionInBytes, 0 , functionInBytesLength)
92
107
93
- private [hive] case class HiveSimpleUdf (functionClassName : String , children : Seq [Expression ])
94
- extends HiveUdf with HiveInspectors {
108
+ // deserialize the function object via Hive Utilities
109
+ instance = HiveShim .deserializePlan(new java.io.ByteArrayInputStream (functionInBytes),
110
+ getContextOrSparkClassLoader.loadClass(functionClassName))
111
+ }
112
+
113
+ def createFunction [UDFType ]() = {
114
+ if (instance == null ) {
115
+ instance = getContextOrSparkClassLoader.loadClass(functionClassName).newInstance
116
+ }
117
+ instance.asInstanceOf [UDFType ]
118
+ }
119
+ }
95
120
121
+ private [hive] case class HiveSimpleUdf (cache : HiveFunctionCache , children : Seq [Expression ])
122
+ extends Expression with HiveInspectors with Logging {
123
+ type EvaluatedType = Any
96
124
type UDFType = UDF
97
125
126
+ def nullable = true
127
+
128
+ @ transient
129
+ lazy val function = cache.createFunction[UDFType ]()
130
+
98
131
@ transient
99
132
protected lazy val method =
100
133
function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo))
@@ -131,6 +164,8 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[
131
164
.convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _* ): _* ),
132
165
returnInspector)
133
166
}
167
+
168
+ override def toString = s " $nodeName# ${cache.functionClassName}( ${children.mkString(" ," )}) "
134
169
}
135
170
136
171
// Adapter from Catalyst ExpressionResult to Hive DeferredObject
@@ -144,16 +179,23 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector)
144
179
override def get (): AnyRef = wrap(func(), oi)
145
180
}
146
181
147
- private [hive] case class HiveGenericUdf (functionClassName : String , children : Seq [Expression ])
148
- extends HiveUdf with HiveInspectors {
182
+ private [hive] case class HiveGenericUdf (cache : HiveFunctionCache , children : Seq [Expression ])
183
+ extends Expression with HiveInspectors with Logging {
149
184
type UDFType = GenericUDF
185
+ type EvaluatedType = Any
186
+
187
+ def nullable = true
188
+
189
+ @ transient
190
+ lazy val function = cache.createFunction[UDFType ]()
150
191
151
192
@ transient
152
193
protected lazy val argumentInspectors = children.map(toInspector)
153
194
154
195
@ transient
155
- protected lazy val returnInspector =
196
+ protected lazy val returnInspector = {
156
197
function.initializeAndFoldConstants(argumentInspectors.toArray)
198
+ }
157
199
158
200
@ transient
159
201
protected lazy val isUDFDeterministic = {
@@ -183,18 +225,19 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq
183
225
}
184
226
unwrap(function.evaluate(deferedObjects), returnInspector)
185
227
}
228
+
229
+ override def toString = s " $nodeName# ${cache.functionClassName}( ${children.mkString(" ," )}) "
186
230
}
187
231
188
232
private [hive] case class HiveGenericUdaf (
189
- functionClassName : String ,
233
+ cache : HiveFunctionCache ,
190
234
children : Seq [Expression ]) extends AggregateExpression
191
- with HiveInspectors
192
- with HiveFunctionFactory {
235
+ with HiveInspectors {
193
236
194
237
type UDFType = AbstractGenericUDAFResolver
195
238
196
239
@ transient
197
- protected lazy val resolver : AbstractGenericUDAFResolver = createFunction()
240
+ protected lazy val resolver : AbstractGenericUDAFResolver = cache. createFunction()
198
241
199
242
@ transient
200
243
protected lazy val objectInspector = {
@@ -209,22 +252,22 @@ private[hive] case class HiveGenericUdaf(
209
252
210
253
def nullable : Boolean = true
211
254
212
- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
255
+ override def toString = s " $nodeName# ${cache. functionClassName} ( ${children.mkString(" ," )}) "
213
256
214
- def newInstance () = new HiveUdafFunction (functionClassName , children, this )
257
+ def newInstance () = new HiveUdafFunction (cache , children, this )
215
258
}
216
259
217
260
/** It is used as a wrapper for the hive functions which uses UDAF interface */
218
261
private [hive] case class HiveUdaf (
219
- functionClassName : String ,
262
+ cache : HiveFunctionCache ,
220
263
children : Seq [Expression ]) extends AggregateExpression
221
- with HiveInspectors
222
- with HiveFunctionFactory {
264
+ with HiveInspectors {
223
265
224
266
type UDFType = UDAF
225
267
226
268
@ transient
227
- protected lazy val resolver : AbstractGenericUDAFResolver = new GenericUDAFBridge (createFunction())
269
+ protected lazy val resolver : AbstractGenericUDAFResolver =
270
+ new GenericUDAFBridge (cache.createFunction())
228
271
229
272
@ transient
230
273
protected lazy val objectInspector = {
@@ -239,10 +282,10 @@ private[hive] case class HiveUdaf(
239
282
240
283
def nullable : Boolean = true
241
284
242
- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
285
+ override def toString = s " $nodeName# ${cache. functionClassName} ( ${children.mkString(" ," )}) "
243
286
244
287
def newInstance () =
245
- new HiveUdafFunction (functionClassName , children, this , true )
288
+ new HiveUdafFunction (cache , children, this , true )
246
289
}
247
290
248
291
/**
@@ -257,13 +300,13 @@ private[hive] case class HiveUdaf(
257
300
* user defined aggregations, which have clean semantics even in a partitioned execution.
258
301
*/
259
302
private [hive] case class HiveGenericUdtf (
260
- functionClassName : String ,
303
+ cache : HiveFunctionCache ,
261
304
aliasNames : Seq [String ],
262
305
children : Seq [Expression ])
263
- extends Generator with HiveInspectors with HiveFunctionFactory {
306
+ extends Generator with HiveInspectors {
264
307
265
308
@ transient
266
- protected lazy val function : GenericUDTF = createFunction()
309
+ protected lazy val function : GenericUDTF = cache. createFunction()
267
310
268
311
@ transient
269
312
protected lazy val inputInspectors = children.map(_.dataType).map(toInspector)
@@ -320,25 +363,24 @@ private[hive] case class HiveGenericUdtf(
320
363
}
321
364
}
322
365
323
- override def toString = s " $nodeName# $functionClassName( ${children.mkString(" ," )}) "
366
+ override def toString = s " $nodeName# ${cache. functionClassName} ( ${children.mkString(" ," )}) "
324
367
}
325
368
326
369
private [hive] case class HiveUdafFunction (
327
- functionClassName : String ,
370
+ cache : HiveFunctionCache ,
328
371
exprs : Seq [Expression ],
329
372
base : AggregateExpression ,
330
373
isUDAFBridgeRequired : Boolean = false )
331
374
extends AggregateFunction
332
- with HiveInspectors
333
- with HiveFunctionFactory {
375
+ with HiveInspectors {
334
376
335
377
def this () = this (null , null , null )
336
378
337
379
private val resolver =
338
380
if (isUDAFBridgeRequired) {
339
- new GenericUDAFBridge (createFunction[UDAF ]())
381
+ new GenericUDAFBridge (cache. createFunction[UDAF ]())
340
382
} else {
341
- createFunction[AbstractGenericUDAFResolver ]()
383
+ cache. createFunction[AbstractGenericUDAFResolver ]()
342
384
}
343
385
344
386
private val inspectors = exprs.map(_.dataType).map(toInspector).toArray
@@ -361,3 +403,4 @@ private[hive] case class HiveUdafFunction(
361
403
function.iterate(buffer, inputs)
362
404
}
363
405
}
406
+
0 commit comments