@PublicEvolving @Retention(value=RUNTIME) @Target(value={TYPE,METHOD,FIELD,PARAMETER}) public @interface DataTypeHint
DataType
.
Data type hints can parameterize or replace the default extraction logic of individual function parameters and return types, structured classes, or fields of structured classes. An implementer can choose to what extent the default extraction logic should be modified.
The following examples show how to explicitly specify data types, how to parameterize the extraction logic, or how to accept any data type as an input data type:
@DataTypeHint("INT")
defines an INT data type with a default conversion class.
@DataTypeHint(value = "TIMESTAMP(3)", bridgedTo = java.sql.Timestamp.class)
defines a
TIMESTAMP data type of millisecond precision with an explicit conversion class.
@DataTypeHint(value = "RAW", bridgedTo = MyCustomClass.class)
defines a RAW data type
with Flink's default serializer for class MyCustomClass
.
@DataTypeHint(value = "RAW", rawSerializer = MyCustomSerializer.class)
defines a RAW
data type with a custom serializer class.
@DataTypeHint(version = V1, allowRawGlobally = TRUE)
parameterizes the extraction by
requesting a extraction logic version of 1 and allowing the RAW data type in this structured type
(and possibly nested fields).
@DataTypeHint(bridgedTo = MyPojo.class, allowRawGlobally = TRUE)
defines that a type
should be extracted from the given conversion class but with parameterized extraction for
allowing RAW types.
@DataTypeHint(inputGroup = ANY)
defines that the input validation should accept any
data type.
Note: All hint parameters are optional. Hint parameters defined on top of a structured type
are inherited by all (deeply) nested fields unless annotated differently. For example, all
occurrences of BigDecimal
will be extracted as DECIMAL(12, 2)
if the
enclosing structured class is annotated with @DataTypeHint(defaultDecimalPrecision = 12,
defaultDecimalScale = 2)
. Individual field annotations allow to deviate from those default
values.
A data type hint on top of a table or aggregate function is similar to defining FunctionHint.output()
for the output type of the function.
FunctionHint
Modifier and Type | Optional Element and Description |
---|---|
HintFlag |
allowRawGlobally
Defines that a RAW data type may be used for all classes that cannot be mapped to any
SQL-like data type or cause an error.
|
String[] |
allowRawPattern
Defines that a RAW data type may be used for all classes that cannot be mapped to any
SQL-like data type or cause an error if their class name starts with or is equal to one of
the given patterns.
|
Class<?> |
bridgedTo
Adds a hint that data should be represented using the given class when entering or leaving
the table ecosystem.
|
int |
defaultDecimalPrecision
Defines a default precision for all decimal data types that are extracted.
|
int |
defaultDecimalScale
Defines a default scale for all decimal data types that are extracted.
|
int |
defaultSecondPrecision
Defines a default fractional second precision for all day-time intervals and timestamps that
are extracted.
|
int |
defaultYearPrecision
Defines a default year precision for all year-month intervals that are extracted.
|
String[] |
forceRawPattern
Defines that a RAW data type must be used for all classes if their class name starts with or
is equal to one of the given patterns.
|
InputGroup |
inputGroup
This hint parameter influences the extraction of a
TypeInference in functions. |
Class<? extends TypeSerializer<?>> |
rawSerializer
Adds a hint that defines a custom serializer that should be used for serializing and
deserializing opaque RAW types.
|
String |
value
The explicit string representation of a data type.
|
ExtractionVersion |
version
Version that describes the expected behavior of the reflection-based data type extraction.
|
public abstract String value
DataTypes
for a list of
supported data types. For example, INT
for an integer data type or DECIMAL(12,
5)
for decimal data type with precision 12 and scale 5.
Use an unparameterized RAW
string for explicitly declaring an opaque data type
without entering a full type string. For Flink's default RAW serializer, use
@DataTypeHint("RAW")
or more specific @DataTypeHint(value = "RAW", bridgedTo =
MyCustomClass.class)
. For a custom RAW serializer, use @DataTypeHint(value = "RAW",
rawSerializer = MyCustomSerializer.class)
.
By default, the empty string represents an undefined data type which means that it will be derived automatically.
Use inputGroup()
for accepting a group of similar data types if this hint is used
to enrich input arguments.
LogicalType.asSerializableString()
,
DataTypes
public abstract Class<?> bridgedTo
If an explicit data type has been defined via value()
, a supported conversion
class depends on the logical type and its nullability property.
If an explicit data type has not been defined via value()
, this class is used for
reflective extraction of a data type.
Please see the implementation of LogicalType.supportsInputConversion(Class)
,
LogicalType.supportsOutputConversion(Class)
, or the documentation for more
information about supported conversions.
By default, the conversion class is reflectively extracted.
AbstractDataType.bridgedTo(Class)
public abstract Class<? extends TypeSerializer<?>> rawSerializer
value()
is explicitly defined as an
unparameterized RAW
string or if (possibly nested) fields in a structured type need
to be handled as an opaque type.
By default, Flink's default RAW serializer is used.
DataTypes.RAW(Class, TypeSerializer)
public abstract InputGroup inputGroup
TypeInference
in functions. It
adds a hint for accepting pre-defined groups of similar types, i.e., more than just one
explicit data type.
Note: This hint parameter is only interpreted when used in function hints or next to arguments of implementation methods. It has highest precedence above all other hint parameter.
Some examples:
{@code // expects an integer for the first input argument and allows any data type for the second
public abstract ExtractionVersion version
It is meant for future backward compatibility. Whenever the extraction logic is changed, old function and structured type classes should still return the same data type as before when versioned accordingly.
By default, the version is always the most recent one.
public abstract HintFlag allowRawGlobally
By default, this parameter is set to false
which means that an exception is thrown
for unmapped types. This is helpful to identify and fix faulty implementations. It is
generally recommended to use SQL-like types instead of enabling RAW opaque types.
If RAW types cannot be avoided, they should be enabled only in designated areas (i.e.,
within package prefixes using allowRawPattern()
) in order to not swallow all errors.
However, this parameter globally enables RAW types for the annotated class and all nested
fields.
This parameter has higher precedence than allowRawPattern()
.
DataTypes.RAW(Class, TypeSerializer)
public abstract String[] allowRawPattern
For example, if some Joda time classes cannot be mapped to any SQL-like data type, one can
define the class prefix "org.joda.time"
. Some classes might be handled as structured
types on a best effort basis but others will be RAW data types if necessary.
By default, the pattern list is empty which means that an exception is thrown for unmapped types. This is helpful to identify and fix faulty implementations. It is generally recommended to use SQL-like types instead of enabling RAW opaque types.
If RAW types cannot be avoided, this parameter should be used to enabled them only in designated areas (i.e., within package prefixes) in order to not swallow all errors.
This parameter has lower precedence than allowRawGlobally()
which would globally
allow RAW types in the annotated class and all nested fields.
DataTypes.RAW(Class, TypeSerializer)
public abstract String[] forceRawPattern
For example, one can define the class prefix "org.joda.time",
"java.math.BigDecimal"
which means that all Joda time classes and Java's BigDecimal
will be handled as RAW data types regardless if they could be mapped to
a more SQL-like data type.
By default, the pattern list is empty which means that an exception is thrown for unmapped types. This is helpful to identify and fix faulty implementations. It is generally recommended to use SQL-like types instead of enabling RAW opaque types.
If RAW types cannot be avoided, they should be enabled only in designated areas (i.e.,
within package prefixes) in order to not swallow all errors. However, compared to allowRawPattern()
, this parameter forces to skip the extraction entirely for the given
prefixes instead of trying to match a class to a more SQL-like data type.
This parameter has the highest precedence of all data type related hint parameters.
DataTypes.RAW(Class, TypeSerializer)
public abstract int defaultDecimalPrecision
By default, decimals are not extracted from classes such as BigDecimal
because they don't define a fixed precision and scale which is required in the SQL type
system.
public abstract int defaultDecimalScale
By default, decimals are not extracted from classes such as BigDecimal
because they don't define a fixed precision and scale which is required in the SQL type
system.
public abstract int defaultYearPrecision
0
, an INTERVAL MONTH
data type is extracted.
By default, INTERVAL YEAR(4) TO MONTH
data types are extracted from classes such
as Period
.
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.