Class Expressions
- java.lang.Object
-
- org.apache.flink.table.api.Expressions
-
@PublicEvolving public final class Expressions extends Object
Entry point of the Table API Expression DSL such as:$("myField").plus(10).abs()
This class contains static methods for referencing table columns, creating literals, and building more complex
Expression
chains.ApiExpressions
are pure API entities that are further translated intoResolvedExpressions
under the hood.For fluent definition of expressions and easier readability, we recommend adding a star import to the methods of this class:
import static org.apache.flink.table.api.Expressions.*;
Check the documentation for more programming language specific APIs, for example, by using Scala implicits.
-
-
Field Summary
Fields Modifier and Type Field Description static ApiExpression
CURRENT_RANGE
Offset constant to be used in thefollowing
clause ofOver
windows.static ApiExpression
CURRENT_ROW
Offset constant to be used in thefollowing
clause ofOver
windows.static ApiExpression
UNBOUNDED_RANGE
Offset constant to be used in thepreceding
clause of unboundedOver
windows.static ApiExpression
UNBOUNDED_ROW
Offset constant to be used in thepreceding
clause of unboundedOver
windows.
-
Constructor Summary
Constructors Constructor Description Expressions()
-
Method Summary
All Methods Static Methods Concrete Methods Deprecated Methods Modifier and Type Method Description static ApiExpression
$(String name)
Creates an unresolved reference to a table's column.static ApiExpression
and(Object predicate0, Object predicate1, Object... predicates)
Boolean AND in three-valued logic.static ApiExpression
array(Object head, Object... tail)
Creates an array of literals.static ApiExpression
atan2(Object y, Object x)
Calculates the arc tangent of a given coordinate.static ApiExpression
call(Class<? extends UserDefinedFunction> function, Object... arguments)
A call to an unregistered, inline function.static ApiExpression
call(String path, Object... arguments)
A call to a function that will be looked up in a catalog.static ApiExpression
call(UserDefinedFunction function, Object... arguments)
A call to an unregistered, inline function.static ApiExpression
callSql(String sqlExpression)
A call to a SQL expression.static ApiExpression
coalesce(Object... args)
Returns the first argument that is not NULL.static ApiExpression
col(String name)
Creates an unresolved reference to a table's column.static ApiExpression
concat(Object string, Object... strings)
Returns the string that results from concatenating the arguments.static ApiExpression
concatWs(Object separator, Object string, Object... strings)
Returns the string that results from concatenating the arguments and separator.static ApiExpression
convertTz(Object dateStr, Object tzFrom, Object tzTo)
Converts a datetime dateStr (with default ISO timestamp format 'yyyy-MM-dd HH:mm:ss') from time zone tzFrom to time zone tzTo.static ApiExpression
currentDatabase()
Return the current database, the return type of this expression isDataTypes.STRING()
.static ApiExpression
currentDate()
Returns the current SQL date in local time zone, the return type of this expression isDataTypes.DATE()
.static ApiExpression
currentTime()
Returns the current SQL time in local time zone, the return type of this expression isDataTypes.TIME()
.static ApiExpression
currentTimestamp()
Returns the current SQL timestamp in local time zone, the return type of this expression isDataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()
.static ApiExpression
currentWatermark(Object rowtimeAttribute)
Returns the current watermark for the given rowtime attribute, orNULL
if no common watermark of all upstream operations is available at the current operation in the pipeline.static ApiExpression
dateFormat(Object timestamp, Object format)
Formats a timestamp as a string using a specified format.static ApiExpression
e()
Returns a value that is closer than any other value to e.static ApiExpression
fromUnixtime(Object unixtime)
Converts unix timestamp (seconds since '1970-01-01 00:00:00' UTC) to datetime string in the "yyyy-MM-dd HH:mm:ss" format.static ApiExpression
fromUnixtime(Object unixtime, Object format)
Converts unix timestamp (seconds since '1970-01-01 00:00:00' UTC) to datetime string in the given format.static ApiExpression
ifThenElse(Object condition, Object ifTrue, Object ifFalse)
Ternary conditional operator that decides which of two other expressions should be evaluated based on a evaluated boolean condition.static ApiExpression
jsonArray(JsonOnNull onNull, Object... values)
Builds a JSON array string from a list of values.static ApiExpression
jsonArrayAgg(JsonOnNull onNull, Object itemExpr)
Builds a JSON object string by aggregating items into an array.static ApiExpression
jsonObject(JsonOnNull onNull, Object... keyValues)
Builds a JSON object string from a list of key-value pairs.static ApiExpression
jsonObjectAgg(JsonOnNull onNull, Object keyExpr, Object valueExpr)
Builds a JSON object string by aggregating key-value expressions into a single JSON object.static ApiExpression
jsonString(Object value)
Serializes a value into JSON.static ApiExpression
lit(Object v)
Creates a SQL literal.static ApiExpression
lit(Object v, DataType dataType)
Creates a SQL literal of a givenDataType
.static ApiExpression
localTime()
Returns the current SQL time in local time zone, the return type of this expression isDataTypes.TIME()
, this is a synonym forcurrentTime()
.static ApiExpression
localTimestamp()
Returns the current SQL timestamp in local time zone, the return type of this expression isDataTypes.TIMESTAMP()
.static ApiExpression
log(Object value)
Calculates the logarithm of the given value.static ApiExpression
log(Object base, Object value)
Calculates the logarithm of the given value to the given base.static ApiExpression
map(Object key, Object value, Object... tail)
Creates a map of expressions.static ApiExpression
mapFromArrays(Object key, Object value)
Creates a map from an array of keys and an array of values.static ApiExpression
negative(Object v)
Returns negative numeric.static ApiExpression
not(Object expression)
Inverts a given boolean expression.static ApiExpression
nullOf(TypeInformation<?> typeInfo)
Deprecated.This method will be removed in future versions as it uses the old type system.static ApiExpression
nullOf(DataType dataType)
Returns a null literal value of a given data type.static ApiExpression
or(Object predicate0, Object predicate1, Object... predicates)
Boolean OR in three-valued logic.static ApiExpression
pi()
Returns a value that is closer than any other value to pi.static ApiExpression
rand()
Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive).static ApiExpression
rand(Object seed)
Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive) with a initial seed.static ApiExpression
randInteger(Object bound)
Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive).static ApiExpression
randInteger(Object seed, Object bound)
Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive) with a initial seed.static ApiExpression
range(int start, int end)
Indicates an index based range, which can be used in columns selection.static ApiExpression
range(String start, String end)
Indicates a range from 'start' to 'end', which can be used in columns selection.static ApiExpression
row(Object head, Object... tail)
Creates a row of expressions.static ApiExpression
rowInterval(Long rows)
Creates an interval of rows.static ApiExpression
sourceWatermark()
Source watermark declaration forSchema
.static ApiExpression
temporalOverlaps(Object leftTimePoint, Object leftTemporal, Object rightTimePoint, Object rightTemporal)
Determines whether two anchored time intervals overlap.static ApiExpression
timestampDiff(TimePointUnit timePointUnit, Object timePoint1, Object timePoint2)
Returns the (signed) number ofTimePointUnit
between timePoint1 and timePoint2.static ApiExpression
toDate(Object dateStr)
Converts the given date string with format 'yyyy-MM-dd' toDataTypes.DATE()
.static ApiExpression
toDate(Object dateStr, Object format)
Converts the date string with the specified format toDataTypes.DATE()
.static ApiExpression
toTimestamp(Object timestampStr)
Converts the given date time string with format 'yyyy-MM-dd HH:mm:ss' under the 'UTC+0' time zone toDataTypes.TIMESTAMP()
.static ApiExpression
toTimestamp(Object timestampStr, Object format)
Converts the given time string with the specified format under the 'UTC+0' time zone toDataTypes.TIMESTAMP()
.static ApiExpression
toTimestampLtz(Object numericEpochTime, Object precision)
Converts a numeric type epoch time toDataTypes.TIMESTAMP_LTZ(int)
.static ApiExpression
unixTimestamp()
Gets the current unix timestamp in seconds.static ApiExpression
unixTimestamp(Object timestampStr)
Converts the given date time string with format 'yyyy-MM-dd HH:mm:ss' to unix timestamp (in seconds), using the time zone specified in the table config.static ApiExpression
unixTimestamp(Object timestampStr, Object format)
Converts the given date time string with the specified format to unix timestamp (in seconds), using the specified timezone in table config.static ApiExpression
uuid()
Returns an UUID (Universally Unique Identifier) string (e.g., "3d3c68f7-f608-473f-b60c-b0c44ad4cc4e") according to RFC 4122 type 4 (pseudo randomly generated) UUID.static ApiExpression
withAllColumns()
Creates an expression that selects all columns.static ApiExpression
withColumns(Object head, Object... tail)
Creates an expression that selects a range of columns.static ApiExpression
withoutColumns(Object head, Object... tail)
Creates an expression that selects all columns except for the given range of columns.
-
-
-
Field Detail
-
UNBOUNDED_ROW
public static final ApiExpression UNBOUNDED_ROW
Offset constant to be used in thepreceding
clause of unboundedOver
windows. Use this constant for a time interval. Unbounded over windows start with the first row of a partition.
-
UNBOUNDED_RANGE
public static final ApiExpression UNBOUNDED_RANGE
Offset constant to be used in thepreceding
clause of unboundedOver
windows. Use this constant for a row-count interval. Unbounded over windows start with the first row of a partition.
-
CURRENT_ROW
public static final ApiExpression CURRENT_ROW
Offset constant to be used in thefollowing
clause ofOver
windows. Use this for setting the upper bound of the window to the current row.
-
CURRENT_RANGE
public static final ApiExpression CURRENT_RANGE
Offset constant to be used in thefollowing
clause ofOver
windows. Use this for setting the upper bound of the window to the sort key of the current row, i.e., all rows with the same sort key as the current row are included in the window.
-
-
Method Detail
-
$
public static ApiExpression $(String name)
Creates an unresolved reference to a table's column.Example:
tab.select($("key"), $("value"))
- See Also:
col(String)
,withAllColumns()
-
col
public static ApiExpression col(String name)
Creates an unresolved reference to a table's column.Because
$(String)
is not supported by every JVM language due to the dollar sign, this method provides a synonym with the same behavior.Example:
tab.select(col("key"), col("value"))
- See Also:
withAllColumns()
-
lit
public static ApiExpression lit(Object v)
Creates a SQL literal.The data type is derived from the object's class and its value.
For example:
lit(12)
leads toINT
lit("abc")
leads toCHAR(3)
lit(new BigDecimal("123.45"))
leads toDECIMAL(5, 2)
See
ValueDataTypeConverter
for a list of supported literal values.
-
lit
public static ApiExpression lit(Object v, DataType dataType)
Creates a SQL literal of a givenDataType
.The method
lit(Object)
is preferred as it extracts theDataType
automatically. Use this method only when necessary. The class ofv
must be supported according to theLogicalType.supportsInputConversion(Class)
.
-
range
public static ApiExpression range(String start, String end)
Indicates a range from 'start' to 'end', which can be used in columns selection.Example:
Table table = ... table.select(withColumns(range(b, c)))
-
range
public static ApiExpression range(int start, int end)
Indicates an index based range, which can be used in columns selection.Example:
Table table = ... table.select(withColumns(range(3, 4)))
-
and
public static ApiExpression and(Object predicate0, Object predicate1, Object... predicates)
Boolean AND in three-valued logic.
-
or
public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates)
Boolean OR in three-valued logic.
-
not
public static ApiExpression not(Object expression)
Inverts a given boolean expression.This method supports a three-valued logic by preserving
NULL
. This means if the input expression isNULL
, the result will also beNULL
.The resulting type is nullable if and only if the input type is nullable.
Examples:
not(lit(true)) // false not(lit(false)) // true not(lit(null, DataTypes.BOOLEAN())) // null
-
currentDate
public static ApiExpression currentDate()
Returns the current SQL date in local time zone, the return type of this expression isDataTypes.DATE()
.
-
currentTime
public static ApiExpression currentTime()
Returns the current SQL time in local time zone, the return type of this expression isDataTypes.TIME()
.
-
currentTimestamp
public static ApiExpression currentTimestamp()
Returns the current SQL timestamp in local time zone, the return type of this expression isDataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()
.
-
currentWatermark
public static ApiExpression currentWatermark(Object rowtimeAttribute)
Returns the current watermark for the given rowtime attribute, orNULL
if no common watermark of all upstream operations is available at the current operation in the pipeline.The function returns the watermark with the same type as the rowtime attribute, but with an adjusted precision of 3. For example, if the rowtime attribute is
TIMESTAMP_LTZ(9)
, the function will returnTIMESTAMP_LTZ(3)
.If no watermark has been emitted yet, the function will return
NULL
. Users must take care of this when comparing against it, e.g. in order to filter out late data you can useWHERE CURRENT_WATERMARK(ts) IS NULL OR ts > CURRENT_WATERMARK(ts)
-
currentDatabase
public static ApiExpression currentDatabase()
Return the current database, the return type of this expression isDataTypes.STRING()
.
-
localTime
public static ApiExpression localTime()
Returns the current SQL time in local time zone, the return type of this expression isDataTypes.TIME()
, this is a synonym forcurrentTime()
.
-
localTimestamp
public static ApiExpression localTimestamp()
Returns the current SQL timestamp in local time zone, the return type of this expression isDataTypes.TIMESTAMP()
.
-
toDate
public static ApiExpression toDate(Object dateStr)
Converts the given date string with format 'yyyy-MM-dd' toDataTypes.DATE()
.- Parameters:
dateStr
- The date string.- Returns:
- The date value of
DataTypes.DATE()
type.
-
toDate
public static ApiExpression toDate(Object dateStr, Object format)
Converts the date string with the specified format toDataTypes.DATE()
.- Parameters:
dateStr
- The date string.format
- The format of the string.- Returns:
- The date value of
DataTypes.DATE()
type.
-
toTimestamp
public static ApiExpression toTimestamp(Object timestampStr)
Converts the given date time string with format 'yyyy-MM-dd HH:mm:ss' under the 'UTC+0' time zone toDataTypes.TIMESTAMP()
.- Parameters:
timestampStr
- The date time string.- Returns:
- The timestamp value with
DataTypes.TIMESTAMP()
type.
-
toTimestamp
public static ApiExpression toTimestamp(Object timestampStr, Object format)
Converts the given time string with the specified format under the 'UTC+0' time zone toDataTypes.TIMESTAMP()
.- Parameters:
timestampStr
- The date time string.format
- The format of the string.- Returns:
- The timestamp value with
DataTypes.TIMESTAMP()
type.
-
toTimestampLtz
public static ApiExpression toTimestampLtz(Object numericEpochTime, Object precision)
Converts a numeric type epoch time toDataTypes.TIMESTAMP_LTZ(int)
.The supported precision is 0 or 3:
- 0 means the numericEpochTime is in second.
- 3 means the numericEpochTime is in millisecond.
- Parameters:
numericEpochTime
- The epoch time with numeric type.precision
- The precision to indicate the epoch time is in second or millisecond.- Returns:
- The timestamp value with
DataTypes.TIMESTAMP_LTZ(int)
type.
-
temporalOverlaps
public static ApiExpression temporalOverlaps(Object leftTimePoint, Object leftTemporal, Object rightTimePoint, Object rightTemporal)
Determines whether two anchored time intervals overlap. Time point and temporal are transformed into a range defined by two time points (start, end). The function evaluatesleftEnd >= rightStart && rightEnd >= leftStart
.It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
e.g.
temporalOverlaps( lit("2:55:00").toTime(), lit(1).hours(), lit("3:30:00").toTime(), lit(2).hours() )
leads to true
-
dateFormat
public static ApiExpression dateFormat(Object timestamp, Object format)
Formats a timestamp as a string using a specified format. The format must be compatible with MySQL's date formatting syntax as used by the date_parse function.For example
dataFormat($("time"), "%Y, %d %M")
results in strings formatted as "2017, 05 May".- Parameters:
timestamp
- The timestamp to format as string.format
- The format of the string.- Returns:
- The formatted timestamp as string.
-
timestampDiff
public static ApiExpression timestampDiff(TimePointUnit timePointUnit, Object timePoint1, Object timePoint2)
Returns the (signed) number ofTimePointUnit
between timePoint1 and timePoint2.For example,
timestampDiff(TimePointUnit.DAY, lit("2016-06-15").toDate(), lit("2016-06-18").toDate()
leads to 3.- Parameters:
timePointUnit
- The unit to compute diff.timePoint1
- The first point in time.timePoint2
- The second point in time.- Returns:
- The number of intervals as integer value.
-
convertTz
public static ApiExpression convertTz(Object dateStr, Object tzFrom, Object tzTo)
Converts a datetime dateStr (with default ISO timestamp format 'yyyy-MM-dd HH:mm:ss') from time zone tzFrom to time zone tzTo. The format of time zone should be either an abbreviation such as "PST", a full name such as "America/Los_Angeles", or a custom ID such as "GMT-08:00". E.g., convertTz('1970-01-01 00:00:00', 'UTC', 'America/Los_Angeles') returns '1969-12-31 16:00:00'.- Parameters:
dateStr
- the date time stringtzFrom
- the original time zonetzTo
- the target time zone- Returns:
- The formatted timestamp as string.
-
fromUnixtime
public static ApiExpression fromUnixtime(Object unixtime)
Converts unix timestamp (seconds since '1970-01-01 00:00:00' UTC) to datetime string in the "yyyy-MM-dd HH:mm:ss" format.- Parameters:
unixtime
- The unix timestamp with numeric type.- Returns:
- The formatted timestamp as string.
-
fromUnixtime
public static ApiExpression fromUnixtime(Object unixtime, Object format)
Converts unix timestamp (seconds since '1970-01-01 00:00:00' UTC) to datetime string in the given format.- Parameters:
unixtime
- The unix timestamp with numeric type.format
- The format of the string.- Returns:
- The formatted timestamp as string.
-
unixTimestamp
public static ApiExpression unixTimestamp()
Gets the current unix timestamp in seconds. This function is not deterministic which means the value would be recalculated for each record.- Returns:
- The current unix timestamp as bigint.
-
unixTimestamp
public static ApiExpression unixTimestamp(Object timestampStr)
Converts the given date time string with format 'yyyy-MM-dd HH:mm:ss' to unix timestamp (in seconds), using the time zone specified in the table config.- Parameters:
timestampStr
- The date time string.- Returns:
- The converted timestamp as bigint.
-
unixTimestamp
public static ApiExpression unixTimestamp(Object timestampStr, Object format)
Converts the given date time string with the specified format to unix timestamp (in seconds), using the specified timezone in table config.- Parameters:
timestampStr
- The date time string.format
- The format of the date time string.- Returns:
- The converted timestamp as bigint.
-
array
public static ApiExpression array(Object head, Object... tail)
Creates an array of literals.
-
row
public static ApiExpression row(Object head, Object... tail)
Creates a row of expressions.
-
map
public static ApiExpression map(Object key, Object value, Object... tail)
Creates a map of expressions.table.select( map( "key1", 1, "key2", 2, "key3", 3 ))
Note keys and values should have the same types for all entries.
-
mapFromArrays
public static ApiExpression mapFromArrays(Object key, Object value)
Creates a map from an array of keys and an array of values.table.select( mapFromArrays( array("key1", "key2", "key3"), array(1, 2, 3) ))
Note both arrays should have the same length.
-
rowInterval
public static ApiExpression rowInterval(Long rows)
Creates an interval of rows.
-
pi
public static ApiExpression pi()
Returns a value that is closer than any other value to pi.
-
e
public static ApiExpression e()
Returns a value that is closer than any other value to e.
-
rand
public static ApiExpression rand()
Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive).
-
rand
public static ApiExpression rand(Object seed)
Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive) with a initial seed. Two rand() functions will return identical sequences of numbers if they have same initial seed.
-
randInteger
public static ApiExpression randInteger(Object bound)
Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive).
-
randInteger
public static ApiExpression randInteger(Object seed, Object bound)
Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive) with a initial seed. Two randInteger() functions will return identical sequences of numbers if they have same initial seed and same bound.
-
concat
public static ApiExpression concat(Object string, Object... strings)
Returns the string that results from concatenating the arguments. Returns NULL if any argument is NULL.
-
atan2
public static ApiExpression atan2(Object y, Object x)
Calculates the arc tangent of a given coordinate.
-
negative
public static ApiExpression negative(Object v)
Returns negative numeric.
-
concatWs
public static ApiExpression concatWs(Object separator, Object string, Object... strings)
Returns the string that results from concatenating the arguments and separator. Returns NULL If the separator is NULL.Note: this function does not skip empty strings. However, it does skip any NULL values after the separator argument.
-
uuid
public static ApiExpression uuid()
Returns an UUID (Universally Unique Identifier) string (e.g., "3d3c68f7-f608-473f-b60c-b0c44ad4cc4e") according to RFC 4122 type 4 (pseudo randomly generated) UUID. The UUID is generated using a cryptographically strong pseudo random number generator.
-
nullOf
public static ApiExpression nullOf(DataType dataType)
Returns a null literal value of a given data type.e.g.
nullOf(DataTypes.INT())
-
nullOf
public static ApiExpression nullOf(TypeInformation<?> typeInfo)
Deprecated.This method will be removed in future versions as it uses the old type system. It is recommended to usenullOf(DataType)
instead which uses the new type system based onDataTypes
. Please make sure to use either the old or the new type system consistently to avoid unintended behavior. See the website documentation for more information.
-
log
public static ApiExpression log(Object value)
Calculates the logarithm of the given value.
-
log
public static ApiExpression log(Object base, Object value)
Calculates the logarithm of the given value to the given base.
-
sourceWatermark
public static ApiExpression sourceWatermark()
Source watermark declaration forSchema
.This is a marker function that doesn't have concrete runtime implementation. It can only be used as a single expression in
Schema.Builder.watermark(String, Expression)
. The declaration will be pushed down into a table source that implements theSupportsSourceWatermark
interface. The source will emit system-defined watermarks afterwards.Please check the documentation whether the connector supports source watermarks.
-
ifThenElse
public static ApiExpression ifThenElse(Object condition, Object ifTrue, Object ifFalse)
Ternary conditional operator that decides which of two other expressions should be evaluated based on a evaluated boolean condition.e.g. ifThenElse($("f0") > 5, "A", "B") leads to "A"
- Parameters:
condition
- boolean conditionifTrue
- expression to be evaluated if condition holdsifFalse
- expression to be evaluated if condition does not hold
-
coalesce
public static ApiExpression coalesce(Object... args)
Returns the first argument that is not NULL.If all arguments are NULL, it returns NULL as well. The return type is the least restrictive, common type of all of its arguments. The return type is nullable if all arguments are nullable as well.
Examples:
// Returns "default" coalesce(null, "default") // Returns the first non-null value among f0 and f1, or "default" if f0 and f1 are both null coalesce($("f0"), $("f1"), "default")
- Parameters:
args
- the input expressions.
-
withAllColumns
public static ApiExpression withAllColumns()
Creates an expression that selects all columns. It can be used wherever an array of expression is accepted such as function calls, projections, or groupings.This expression is a synonym of $("*"). It is semantically equal to
SELECT *
in SQL when used in a projection.Example:
tab.select(withAllColumns())
-
withColumns
public static ApiExpression withColumns(Object head, Object... tail)
Creates an expression that selects a range of columns. It can be used wherever an array of expression is accepted such as function calls, projections, or groupings.A range can either be index-based or name-based. Indices start at 1 and boundaries are inclusive.
e.g. withColumns(range("b", "c")) or withColumns($("*"))
-
withoutColumns
public static ApiExpression withoutColumns(Object head, Object... tail)
Creates an expression that selects all columns except for the given range of columns. It can be used wherever an array of expression is accepted such as function calls, projections, or groupings.A range can either be index-based or name-based. Indices start at 1 and boundaries are inclusive.
e.g. withoutColumns(range("b", "c")) or withoutColumns($("c"))
-
jsonObject
public static ApiExpression jsonObject(JsonOnNull onNull, Object... keyValues)
Builds a JSON object string from a list of key-value pairs.{@param keyValues} is an even-numbered list of alternating key/value pairs. Note that keys must be non-
NULL
string literals, while values may be arbitrary expressions.This function returns a JSON string. The
onNull
behavior defines how to treatNULL
values.Values which are created from another JSON construction function call (
jsonObject
,jsonArray
) are inserted directly rather than as a string. This allows building nested JSON structures.Examples:
// {} jsonObject(JsonOnNull.NULL) // "{\"K1\":\"V1\",\"K2\":\"V2\"}" // {"K1":"V1","K2":"V2"} jsonObject(JsonOnNull.NULL, "K1", "V1", "K2", "V2") // Expressions as values jsonObject(JsonOnNull.NULL, "orderNo", $("orderId")) // ON NULL jsonObject(JsonOnNull.NULL, "K1", nullOf(DataTypes.STRING())) // "{\"K1\":null}" jsonObject(JsonOnNull.ABSENT, "K1", nullOf(DataTypes.STRING())) // "{}" // {"K1":{"K2":"V"}} jsonObject(JsonOnNull.NULL, "K1", jsonObject(JsonOnNull.NULL, "K2", "V"))
- See Also:
jsonArray(JsonOnNull, Object...)
-
jsonObjectAgg
public static ApiExpression jsonObjectAgg(JsonOnNull onNull, Object keyExpr, Object valueExpr)
Builds a JSON object string by aggregating key-value expressions into a single JSON object.The key expression must return a non-nullable character string. Value expressions can be arbitrary, including other JSON functions. If a value is
NULL
, theonNull
behavior defines what to do.Note that keys must be unique. If a key occurs multiple times, an error will be thrown.
This function is currently not supported in
OVER
windows.Examples:
// "{\"Apple\":2,\"Banana\":17,\"Orange\":0}" orders.select(jsonObjectAgg(JsonOnNull.NULL, $("product"), $("cnt")))
-
jsonString
public static ApiExpression jsonString(Object value)
Serializes a value into JSON.This function returns a JSON string containing the serialized value. If the value is
null
, the function returnsnull
.Examples:
// null jsonString(nullOf(DataTypes.INT())) jsonString(1) // "1" jsonString(true) // "true" jsonString("Hello, World!") // "\"Hello, World!\"" jsonString(Arrays.asList(1, 2)) // "[1,2]"
-
jsonArray
public static ApiExpression jsonArray(JsonOnNull onNull, Object... values)
Builds a JSON array string from a list of values.This function returns a JSON string. The values can be arbitrary expressions. The
onNull
behavior defines how to treatNULL
values.Elements which are created from another JSON construction function call (
jsonObject
,jsonArray
) are inserted directly rather than as a string. This allows building nested JSON structures.Examples:
// "[]" jsonArray(JsonOnNull.NULL) // "[1,\"2\"]" jsonArray(JsonOnNull.NULL, 1, "2") // Expressions as values jsonArray(JsonOnNull.NULL, $("orderId")) // ON NULL jsonArray(JsonOnNull.NULL, nullOf(DataTypes.STRING())) // "[null]" jsonArray(JsonOnNull.ABSENT, nullOf(DataTypes.STRING())) // "[]" // "[[1]]" jsonArray(JsonOnNull.NULL, jsonArray(JsonOnNull.NULL, 1))
- See Also:
jsonObject(JsonOnNull, Object...)
-
jsonArrayAgg
public static ApiExpression jsonArrayAgg(JsonOnNull onNull, Object itemExpr)
Builds a JSON object string by aggregating items into an array.Item expressions can be arbitrary, including other JSON functions. If a value is
NULL
, theonNull
behavior defines what to do.This function is currently not supported in
OVER
windows, unbounded session windows, or hop windows.Examples:
// "[\"Apple\",\"Banana\",\"Orange\"]" orders.select(jsonArrayAgg(JsonOnNull.NULL, $("product")))
-
call
public static ApiExpression call(String path, Object... arguments)
A call to a function that will be looked up in a catalog. There are two kinds of functions:- System functions - which are identified with one part names
- Catalog functions - which are identified always with three parts names (catalog, database, function)
Moreover each function can either be a temporary function or permanent one (which is stored in an external catalog).
Based on that two properties the resolution order for looking up a function based on the provided
functionName
is following:- Temporary system function
- System function
- Temporary catalog function
- Catalog function
- See Also:
TableEnvironment.useCatalog(String)
,TableEnvironment.useDatabase(String)
,TableEnvironment.createTemporaryFunction(java.lang.String, java.lang.Class<? extends org.apache.flink.table.functions.UserDefinedFunction>)
,TableEnvironment.createTemporarySystemFunction(java.lang.String, java.lang.Class<? extends org.apache.flink.table.functions.UserDefinedFunction>)
-
call
public static ApiExpression call(UserDefinedFunction function, Object... arguments)
A call to an unregistered, inline function.For functions that have been registered before and are identified by a name, use
call(String, Object...)
.
-
call
public static ApiExpression call(Class<? extends UserDefinedFunction> function, Object... arguments)
A call to an unregistered, inline function.For functions that have been registered before and are identified by a name, use
call(String, Object...)
.
-
callSql
public static ApiExpression callSql(String sqlExpression)
A call to a SQL expression.The given string is parsed and translated into an
Expression
during planning. Only the translated expression is evaluated during runtime.Note: Currently, calls are limited to simple scalar expressions. Calls to aggregate or table-valued functions are not supported. Sub-queries are also not allowed.
-
-