R
- the JVM type of result values, MUST be consistent with the DataType
returned via BoundFunction.resultType()
, according to the mapping above.@Evolving public interface ScalarFunction<R> extends BoundFunction
To evaluate each input row, Spark will first try to lookup and use a "magic method" (described
below) through Java reflection. If the method is not found, Spark will call
produceResult(InternalRow)
as a fallback approach.
The JVM type of result values produced by this function must be the type used by Spark's
InternalRow API for the SQL data type
returned by BoundFunction.resultType()
.
The mapping between DataType
and the corresponding JVM type is defined below.
produceResult(org.apache.spark.sql.catalyst.InternalRow)
throws
UnsupportedOperationException
. Users must choose to either override this method, or
implement a magic method with name MAGIC_METHOD_NAME
, which takes individual parameters
instead of a InternalRow
. The magic method approach is generally recommended because it
provides better performance over the default produceResult(org.apache.spark.sql.catalyst.InternalRow)
, due to optimizations such
as whole-stage codegen, elimination of Java boxing, etc.
The type parameters for the magic method must match those returned from
BoundFunction.inputTypes()
. Otherwise Spark will not be able to find the magic method.
In addition, for stateless Java functions, users can optionally define the
MAGIC_METHOD_NAME
as a static method, which further avoids certain runtime costs such
as Java dynamic dispatch.
For example, a scalar UDF for adding two integers can be defined as follow with the magic method approach:
public class IntegerAdd implements ScalarFunction<Integer>
{
public DataType[] inputTypes() {
return new DataType[] { DataTypes.IntegerType, DataTypes.IntegerType };
}
public int invoke(int left, int right) {
return left + right;
}
}
In the above, since MAGIC_METHOD_NAME
is defined, and also that it has
matching parameter types and return type, Spark will use it to evaluate inputs.
As another example, in the following:
public class IntegerAdd implements ScalarFunction<Integer>
{
public DataType[] inputTypes() {
return new DataType[] { DataTypes.IntegerType, DataTypes.IntegerType };
}
public static int invoke(int left, int right) {
return left + right;
}
public Integer produceResult(InternalRow input) {
return input.getInt(0) + input.getInt(1);
}
}
the class defines both the magic method and the produceResult(org.apache.spark.sql.catalyst.InternalRow)
, and Spark will use
MAGIC_METHOD_NAME
over the produceResult(InternalRow)
as it takes higher
precedence. Also note that the magic method is annotated as a static method in this case.
Resolution on magic method is done during query analysis, where Spark looks up the magic method by first converting the actual input SQL data types to their corresponding Java types following the mapping defined below, and then checking if there is a matching method from all the declared methods in the UDF class, using method name and the Java types.
produceResult(org.apache.spark.sql.catalyst.InternalRow)
approach. With the former, whenever any of the method arguments meet
the following conditions:
produceResult(org.apache.spark.sql.catalyst.InternalRow)
and it is user's responsibility to
handle them in the function implementation.
Because of the difference, if Spark users want to implement special handling of nulls for
nullable primitive arguments, they should override the produceResult(org.apache.spark.sql.catalyst.InternalRow)
method instead
of using the magic method approach.
SQL data type
to Java type which is used
by Spark to infer parameter types for the magic methods as well as return value type for
produceResult(org.apache.spark.sql.catalyst.InternalRow)
:
BooleanType
: boolean
ByteType
: byte
ShortType
: short
IntegerType
: int
LongType
: long
FloatType
: float
DoubleType
: double
StringType
:
UTF8String
DateType
: int
TimestampType
: long
BinaryType
: byte[]
DayTimeIntervalType
: long
YearMonthIntervalType
: int
DecimalType
:
Decimal
StructType
: InternalRow
ArrayType
:
ArrayData
MapType
:
MapData
Modifier and Type | Field and Description |
---|---|
static String |
MAGIC_METHOD_NAME |
Modifier and Type | Method and Description |
---|---|
default R |
produceResult(org.apache.spark.sql.catalyst.InternalRow input)
Applies the function to an input row to produce a value.
|
canonicalName, inputTypes, isDeterministic, isResultNullable, resultType
static final String MAGIC_METHOD_NAME
default R produceResult(org.apache.spark.sql.catalyst.InternalRow input)
input
- an input row