Class ScalarFunction
- java.lang.Object
-
- org.apache.flink.table.functions.UserDefinedFunction
-
- org.apache.flink.table.functions.ScalarFunction
-
- All Implemented Interfaces:
Serializable
,FunctionDefinition
- Direct Known Subclasses:
AddOne
,BuiltInScalarFunction
,GettingStartedExample.AddressNormalizer
,InternalRowMergerFunction
,PythonScalarFunction
,StringRegexReplaceFunction
@PublicEvolving public abstract class ScalarFunction extends UserDefinedFunction
Base class for a user-defined scalar function. A user-defined scalar function maps zero, one, or multiple scalar values to a new scalar value.The behavior of a
ScalarFunction
can be defined by implementing a custom evaluation method. An evaluation method must be declared publicly and namedeval
. Evaluation methods can also be overloaded by implementing multiple methods namedeval
.By default, input and output data types are automatically extracted using reflection. If the reflective information is not sufficient, it can be supported and enriched with
DataTypeHint
andFunctionHint
annotations.The following examples show how to specify a scalar function:
{@code // a function that accepts two INT arguments and computes a sum class SumFunction extends ScalarFunction { public Integer eval(Integer a, Integer b) { return a + b; } } // a function that accepts either INT NOT NULL or BOOLEAN NOT NULL and computes a STRING class StringifyFunction extends ScalarFunction { public String eval(int i) { return String.valueOf(i); } public String eval(boolean b) { return String.valueOf(b); } } // a function that accepts either INT or BOOLEAN and computes a STRING using function hints
- See Also:
- Serialized Form
-
-
Constructor Summary
Constructors Constructor Description ScalarFunction()
-
Method Summary
All Methods Instance Methods Concrete Methods Deprecated Methods Modifier and Type Method Description FunctionKind
getKind()
Returns the kind of function this definition describes.TypeInformation<?>[]
getParameterTypes(Class<?>[] signature)
Deprecated.This method uses the old type system and is based on the old reflective extraction logic.TypeInformation<?>
getResultType(Class<?>[] signature)
Deprecated.This method uses the old type system and is based on the old reflective extraction logic.TypeInference
getTypeInference(DataTypeFactory typeFactory)
Returns the logic for performing type inference of a call to this function definition.-
Methods inherited from class org.apache.flink.table.functions.UserDefinedFunction
close, functionIdentifier, open, toString
-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
-
Methods inherited from interface org.apache.flink.table.functions.FunctionDefinition
getRequirements, isDeterministic, supportsConstantFolding
-
-
-
-
Method Detail
-
getResultType
@Deprecated public TypeInformation<?> getResultType(Class<?>[] signature)
Deprecated.This method uses the old type system and is based on the old reflective extraction logic. The method will be removed in future versions and is only called when using the deprecatedTableEnvironment.registerFunction(...)
method. The new reflective extraction logic (possibly enriched withDataTypeHint
andFunctionHint
) should be powerful enough to cover most use cases. For advanced users, it is possible to overrideUserDefinedFunction.getTypeInference(DataTypeFactory)
.Returns the result type of the evaluation method with a given signature.
-
getParameterTypes
@Deprecated public TypeInformation<?>[] getParameterTypes(Class<?>[] signature)
Deprecated.This method uses the old type system and is based on the old reflective extraction logic. The method will be removed in future versions and is only called when using the deprecatedTableEnvironment.registerFunction(...)
method. The new reflective extraction logic (possibly enriched withDataTypeHint
andFunctionHint
) should be powerful enough to cover most use cases. For advanced users, it is possible to overrideUserDefinedFunction.getTypeInference(DataTypeFactory)
.ReturnsTypeInformation
about the operands of the evaluation method with a given signature.
-
getKind
public final FunctionKind getKind()
Description copied from interface:FunctionDefinition
Returns the kind of function this definition describes.
-
getTypeInference
public TypeInference getTypeInference(DataTypeFactory typeFactory)
Description copied from class:UserDefinedFunction
Returns the logic for performing type inference of a call to this function definition.The type inference process is responsible for inferring unknown types of input arguments, validating input arguments, and producing result types. The type inference process happens independent of a function body. The output of the type inference is used to search for a corresponding runtime implementation.
Instances of type inference can be created by using
TypeInference.newBuilder()
.See
BuiltInFunctionDefinitions
for concrete usage examples.The type inference for user-defined functions is automatically extracted using reflection. It does this by analyzing implementation methods such as
eval() or accumulate()
and the generic parameters of a function class if present. If the reflective information is not sufficient, it can be supported and enriched withDataTypeHint
andFunctionHint
annotations.Note: Overriding this method is only recommended for advanced users. If a custom type inference is specified, it is the responsibility of the implementer to make sure that the output of the type inference process matches with the implementation method:
The implementation method must comply with each
DataType.getConversionClass()
returned by the type inference. For example, ifDataTypes.TIMESTAMP(3).bridgedTo(java.sql.Timestamp.class)
is an expected argument type, the method must accept a calleval(java.sql.Timestamp)
.Regular Java calling semantics (including type widening and autoboxing) are applied when calling an implementation method which means that the signature can be
eval(java.lang.Object)
.The runtime will take care of converting the data to the data format specified by the
DataType.getConversionClass()
coming from the type inference logic.- Specified by:
getTypeInference
in interfaceFunctionDefinition
- Specified by:
getTypeInference
in classUserDefinedFunction
-
-