Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Type alias bug fix & changes based on code review
Dealias collection type before obtaining its companion object
Change collClass to Option
Rename variables
  • Loading branch information
michalsenkyr committed Mar 26, 2017
commit d04e043fcd00204531553cb0a8ac1148d85436f4
Original file line number Diff line number Diff line change
Expand Up @@ -307,11 +307,11 @@ object ScalaReflection extends ScalaReflection {
}
}

val cls = t.companion.decl(TermName("newBuilder")) match {
val cls = t.dealias.companion.decl(TermName("newBuilder")) match {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dealias is not available in scala 2.10, @michalsenkyr can you come up with a workaround? thanks!

case NoSymbol => classOf[Seq[_]]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I checked that the builder for Seq[_] is a mutable.ListBuffer and its build result will be immutable.List. But the original deserialized type of Seq[_] is a WrappedArray.

Can we keep the original expression if we can't find newBuilder here, i.e., returning the WrappedArray?
Or use WrappedArray as the collClass instead of Seq?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rethink about this. It may not be a problem. Let me try to test it first.

case _ => mirror.runtimeClass(t.typeSymbol.asClass)
}
MapObjects(mapFunction, getPath, dataType, cls)
MapObjects(mapFunction, getPath, dataType, Some(cls))

case t if t <:< localTypeOf[Map[_, _]] =>
// TODO: add walked type path for map
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -430,20 +430,21 @@ object MapObjects {
* @param function The function applied on the collection elements.
* @param inputData An expression that when evaluated returns a collection object.
* @param elementType The data type of elements in the collection.
* @param collClass The class of the resulting collection
* @param customCollectionCls Class of the resulting collection (returning ObjectType)
* or None (returning ArrayType)
*/
def apply(
function: Expression => Expression,
inputData: Expression,
elementType: DataType,
collClass: Class[_] = classOf[Array[_]]): MapObjects = {
customCollectionCls: Option[Class[_]] = None): MapObjects = {
val id = curId.getAndIncrement()
val loopValue = s"MapObjects_loopValue$id"
val loopIsNull = s"MapObjects_loopIsNull$id"
val loopVar = LambdaVariable(loopValue, loopIsNull, elementType)
val builderValue = s"MapObjects_builderValue$id"
MapObjects(loopValue, loopIsNull, elementType, function(loopVar), inputData,
collClass, builderValue)
customCollectionCls, builderValue)
}
}

Expand All @@ -453,8 +454,8 @@ object MapObjects {
* function is expressed using catalyst expressions.
*
* The type of the result is determined as follows:
* - ArrayType - when collClass is an array class
* - ObjectType(collClass) - when collClass is a collection class
* - ArrayType - when customCollectionCls is None
* - ObjectType(collection) - when customCollectionCls contains a collection class
*
* The following collection ObjectTypes are currently supported on input:
* Seq, Array, ArrayData, java.util.List
Expand All @@ -468,7 +469,8 @@ object MapObjects {
* @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function
* to handle collection elements.
* @param inputData An expression that when evaluated returns a collection object.
* @param collClass The class of the resulting collection
* @param customCollectionCls Class of the resulting collection (returning ObjectType)
* or None (returning ArrayType)
* @param builderValue The name of the builder variable used to construct the resulting collection
* (used only when returning ObjectType)
*/
Expand All @@ -478,7 +480,7 @@ case class MapObjects private(
loopVarDataType: DataType,
lambdaFunction: Expression,
inputData: Expression,
collClass: Class[_],
customCollectionCls: Option[Class[_]],
builderValue: String) extends Expression with NonSQLExpression {

override def nullable: Boolean = inputData.nullable
Expand All @@ -489,8 +491,8 @@ case class MapObjects private(
throw new UnsupportedOperationException("Only code-generated evaluation is supported")

override def dataType: DataType =
if (!collClass.isArray) ObjectType(collClass)
else ArrayType(lambdaFunction.dataType, containsNull = lambdaFunction.nullable)
customCollectionCls.map(ObjectType.apply).getOrElse(
ArrayType(lambdaFunction.dataType, containsNull = lambdaFunction.nullable))

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val elementJavaType = ctx.javaType(loopVarDataType)
Expand Down Expand Up @@ -573,22 +575,23 @@ case class MapObjects private(
case _ => s"$loopIsNull = $loopValue == null;"
}

val (genInit, genAssign, genResult): (String, String => String, String) =
if (collClass.isArray) {
// array
(s"""$convertedType[] $convertedArray = null;
$convertedArray = $arrayConstructor;""",
genValue => s"$convertedArray[$loopIndex] = $genValue;",
s"new ${classOf[GenericArrayData].getName}($convertedArray);")
} else {
// collection
val collObjectName = s"${collClass.getName}$$.MODULE$$"
val getBuilderVar = s"$collObjectName.newBuilder()"
val (initCollection, addElement, getResult): (String, String => String, String) =
customCollectionCls match {
case Some(cls) =>
// collection
val collObjectName = s"${cls.getName}$$.MODULE$$"
val getBuilderVar = s"$collObjectName.newBuilder()"

(s"""${classOf[Builder[_, _]].getName} $builderValue = $getBuilderVar;
(s"""${classOf[Builder[_, _]].getName} $builderValue = $getBuilderVar;
$builderValue.sizeHint($dataLength);""",
genValue => s"$builderValue.$$plus$$eq($genValue);",
s"(${collClass.getName}) $builderValue.result();")
genValue => s"$builderValue.$$plus$$eq($genValue);",
s"(${cls.getName}) $builderValue.result();")
case None =>
// array
(s"""$convertedType[] $convertedArray = null;
$convertedArray = $arrayConstructor;""",
genValue => s"$convertedArray[$loopIndex] = $genValue;",
s"new ${classOf[GenericArrayData].getName}($convertedArray);")
}

val code = s"""
Expand All @@ -598,7 +601,7 @@ case class MapObjects private(
if (!${genInputData.isNull}) {
$determineCollectionType
int $dataLength = $getLength;
$genInit
$initCollection

int $loopIndex = 0;
while ($loopIndex < $dataLength) {
Expand All @@ -607,15 +610,15 @@ case class MapObjects private(

${genFunction.code}
if (${genFunction.isNull}) {
${genAssign("null")}
${addElement("null")}
} else {
${genAssign(genFunctionValue)}
${addElement(genFunctionValue)}
}

$loopIndex += 1;
}

${ev.value} = $genResult
${ev.value} = $getResult
}
"""
ev.copy(code = code, isNull = genInputData.isNull)
Expand Down