Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,9 @@ private[hive] trait HiveInspectors {
case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType

case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType))

// Hive seems to return this for struct types?
case c: Class[_] if c == classOf[java.lang.Object] => NullType
}

/** Converts hive types to native catalyst types. */
Expand Down Expand Up @@ -147,7 +150,10 @@ private[hive] trait HiveInspectors {
case t: java.sql.Timestamp => t
case s: Seq[_] => seqAsJavaList(s.map(wrap))
case m: Map[_,_] =>
mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) })
// Some UDFs seem to assume we pass in a HashMap.
val hashMap = new java.util.HashMap[AnyRef, AnyRef]()
hashMap.putAll(m.map { case (k, v) => wrap(k) -> wrap(v) })
hashMap
case null => null
}

Expand Down Expand Up @@ -214,6 +220,12 @@ private[hive] trait HiveInspectors {
import TypeInfoFactory._

def toTypeInfo: TypeInfo = dt match {
case ArrayType(elemType, _) =>
getListTypeInfo(elemType.toTypeInfo)
case StructType(fields) =>
getStructTypeInfo(fields.map(_.name), fields.map(_.dataType.toTypeInfo))
case MapType(keyType, valueType, _) =>
getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo)
case BinaryType => binaryTypeInfo
case BooleanType => booleanTypeInfo
case ByteType => byteTypeInfo
Expand Down
41 changes: 24 additions & 17 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,10 @@ private[hive] abstract class HiveFunctionRegistry

HiveSimpleUdf(
functionClassName,
children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) }
children.zip(expectedDataTypes).map {
case (e, NullType) => e
case (e, t) => Cast(e, t)
}
)
} else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
HiveGenericUdf(functionClassName, children)
Expand Down Expand Up @@ -115,22 +118,26 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[
c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head)
}

val constructor = matchingConstructor.getOrElse(
sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}."))

(a: Any) => {
logDebug(
s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.")
// We must make sure that primitives get boxed java style.
if (a == null) {
null
} else {
constructor.newInstance(a match {
case i: Int => i: java.lang.Integer
case bd: BigDecimal => new HiveDecimal(bd.underlying())
case other: AnyRef => other
}).asInstanceOf[AnyRef]
}
matchingConstructor match {
case Some(constructor) =>
(a: Any) => {
logDebug(
s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.")
// We must make sure that primitives get boxed java style.
if (a == null) {
null
} else {
constructor.newInstance(a match {
case i: Int => i: java.lang.Integer
case bd: BigDecimal => new HiveDecimal(bd.underlying())
case other: AnyRef => other
}).asInstanceOf[AnyRef]
}
}
case None =>
(a: Any) => a match {
case wrapper => wrap(wrapper)
}
}
}

Expand Down