Skip to content

Commit

Permalink
fix udt with udf
Browse files Browse the repository at this point in the history
  • Loading branch information
Davies Liu committed Mar 31, 2016
1 parent dd71ba9 commit 8597bba
Show file tree
Hide file tree
Showing 2 changed files with 27 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer
import net.razorvine.pickle.{Pickler, Unpickler}

import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonFunction, PythonRunner}
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRunner}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
Expand Down Expand Up @@ -72,8 +72,6 @@ case class BatchPythonEvaluation(udfs: Seq[PythonUDF], output: Seq[Attribute], c

val (pyFuncs, inputs) = udfs.map(collectFunctions).unzip

// Most of the inputs are primitives, do not use memo for better performance
val pickle = new Pickler(false)
// flatten all the arguments
val allInputs = new ArrayBuffer[Expression]
val dataTypes = new ArrayBuffer[DataType]
Expand All @@ -89,21 +87,30 @@ case class BatchPythonEvaluation(udfs: Seq[PythonUDF], output: Seq[Attribute], c
}.toArray
}.toArray
val projection = newMutableProjection(allInputs, child.output)()
val schema = StructType(dataTypes.map(dt => StructField("", dt)))
val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython)

// enable memo iff we serialize the row with schema (schema and class should be memorized)
val pickle = new Pickler(needConversion)
// Input iterator to Python: input rows are grouped so we send them in batches to Python.
// For each row, add it to the queue.
val inputIterator = iter.grouped(100).map { inputRows =>
val toBePickled = inputRows.map { inputRow =>
queue.add(inputRow)
val row = projection(inputRow)
val fields = new Array[Any](row.numFields)
var i = 0
while (i < row.numFields) {
val dt = dataTypes(i)
fields(i) = EvaluatePython.toJava(row.get(i, dt), dt)
i += 1
if (needConversion) {
EvaluatePython.toJava(row, schema)
} else {
// fast path for these types that does not need conversion in Python
val fields = new Array[Any](row.numFields)
var i = 0
while (i < row.numFields) {
val dt = dataTypes(i)
fields(i) = EvaluatePython.toJava(row.get(i, dt), dt)
i += 1
}
fields
}
fields
}.toArray
pickle.dumps(toBePickled)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,16 @@ object EvaluatePython {
}
}

def needConversionInPython(dt: DataType): Boolean = dt match {
case DateType | TimestampType => true
case _: StructType => true
case _: UserDefinedType[_] => true
case ArrayType(elementType, _) => needConversionInPython(elementType)
case MapType(keyType, valueType, _) =>
needConversionInPython(keyType) || needConversionInPython(valueType)
case _ => false
}

/**
* Helper for converting from Catalyst type to java type suitable for Pyrolite.
*/
Expand Down

0 comments on commit 8597bba

Please sign in to comment.