Class TableConfig
- All Implemented Interfaces:
org.apache.flink.configuration.ReadableConfig,org.apache.flink.configuration.WritableConfig
TableEnvironment session to adjust Table & SQL API
programs.
This class is a pure API class that abstracts configuration from various sources. Currently, configuration can be set in any of the following layers (in the given order):
config.yaml,- CLI parameters,
StreamExecutionEnvironmentwhen bridging to DataStream API,EnvironmentSettings.Builder.withConfiguration(Configuration)/TableEnvironment.create(Configuration),- and
set(ConfigOption, Object)/set(String, String).
The latter two represent the application-specific part of the configuration. They initialize
and directly modify getConfiguration(). Other layers represent the
configuration of the execution context and are immutable.
The getters get(ConfigOption) and getOptional(ConfigOption) give read-only
access to the full configuration. However, application-specific configuration has precedence.
Configuration of outer layers is used for defaults and fallbacks. The setters set(ConfigOption, Object) and set(String, String) will only affect
application-specific configuration.
For common or important configuration options, this class provides getters and setters methods with detailed inline documentation.
For more advanced configuration, users can directly access the underlying key-value map via
getConfiguration(). Users can configure also underlying execution parameters via this
object.
For example:
tEnv.getConfig().addConfiguration(
new Configuration()
.set(CoreOptions.DEFAULT_PARALLELISM, 128)
.set(PipelineOptions.AUTO_WATERMARK_INTERVAL, Duration.ofMillis(800))
.set(CheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(30))
);
Note: Because options are read at different point in time when performing operations, it is recommended to set configuration options early after instantiating a table environment.
-
Method Summary
Modifier and TypeMethodDescriptionvoidaddConfiguration(org.apache.flink.configuration.Configuration configuration) Adds the given key-value configuration to the underlying application-specific configuration.voidaddJobParameter(String key, String value) Sets a custom user parameter that can be accessed viaFunctionContext.getJobParameter(String, String).<T> Tget(org.apache.flink.configuration.ConfigOption<T> option) org.apache.flink.configuration.ConfigurationGives direct access to the underlying application-specific key-value map for advanced configuration.static TableConfigReturns the current session time zone id.Returns the current threshold where generated code will be split into sub-function calls.<T> Optional<T>getOptional(org.apache.flink.configuration.ConfigOption<T> option) Returns the current configuration of Planner for Table API and SQL queries.org.apache.flink.configuration.ReadableConfigGives direct access to the underlying environment-specific key-value map for advanced configuration.Returns the current SQL dialect.Sets an application-specific string-based value for the given string-based key.<T> TableConfigset(org.apache.flink.configuration.ConfigOption<T> option, T value) Sets an application-specific value for the givenConfigOption.voidsetIdleStateRetention(Duration duration) Specifies a retention time interval for how long idle state, i.e., state which was not updated, will be retained.voidsetLocalTimeZone(ZoneId zoneId) Sets the current session time zone id.voidsetMaxGeneratedCodeLength(Integer maxGeneratedCodeLength) Sets current threshold where generated code will be split into sub-function calls.voidsetPlannerConfig(PlannerConfig plannerConfig) Sets the configuration of Planner for Table API and SQL queries.voidsetRootConfiguration(org.apache.flink.configuration.ReadableConfig rootConfiguration) Sets the given configuration asrootConfiguration, which contains any configuration set in the execution context.voidsetSqlDialect(SqlDialect sqlDialect) Sets the current SQL dialect to parse a SQL query.toMap()
-
Method Details
-
set
Sets an application-specific value for the givenConfigOption.This method should be preferred over
set(String, String)as it is type-safe, avoids unnecessary parsing of the value, and provides inline documentation.Note: Scala users might need to convert the value into a boxed type. E.g. by using
Int.box(1)orBoolean.box(false).- Specified by:
setin interfaceorg.apache.flink.configuration.WritableConfig- See Also:
-
set
Sets an application-specific string-based value for the given string-based key.The value will be parsed by the framework on access.
This method exists for convenience when configuring a session with string-based properties. Use
set(ConfigOption, Object)for more type-safety and inline documentation. -
get
public <T> T get(org.apache.flink.configuration.ConfigOption<T> option) This method gives read-only access to the full configuration. However, application-specific configuration has precedence. Configuration of outer layers is used for defaults and fallbacks. See the docs of
TableConfigfor more information.- Specified by:
getin interfaceorg.apache.flink.configuration.ReadableConfig- Type Parameters:
T- type of the value to read- Parameters:
option- metadata of the option to read- Returns:
- read value or
ConfigOption.defaultValue()if not found
-
getOptional
This method gives read-only access to the full configuration. However, application-specific configuration has precedence. Configuration of outer layers is used for defaults and fallbacks. See the docs of
TableConfigfor more information.- Specified by:
getOptionalin interfaceorg.apache.flink.configuration.ReadableConfig- Type Parameters:
T- type of the value to read- Parameters:
option- metadata of the option to read- Returns:
- read value or
Optional.empty()if not found
-
toMap
- Specified by:
toMapin interfaceorg.apache.flink.configuration.ReadableConfig
-
getConfiguration
public org.apache.flink.configuration.Configuration getConfiguration()Gives direct access to the underlying application-specific key-value map for advanced configuration. -
getRootConfiguration
@Internal public org.apache.flink.configuration.ReadableConfig getRootConfiguration()Gives direct access to the underlying environment-specific key-value map for advanced configuration. -
addConfiguration
public void addConfiguration(org.apache.flink.configuration.Configuration configuration) Adds the given key-value configuration to the underlying application-specific configuration. It overwrites existing keys.- Parameters:
configuration- key-value configuration to be added
-
getSqlDialect
Returns the current SQL dialect. -
setSqlDialect
Sets the current SQL dialect to parse a SQL query. Flink's SQL behavior by default. -
getLocalTimeZone
Returns the current session time zone id. It is used when converting to/fromTIMESTAMP WITH LOCAL TIME ZONE. SeesetLocalTimeZone(ZoneId)for more details.- See Also:
-
LocalZonedTimestampType
-
setLocalTimeZone
Sets the current session time zone id. It is used when converting to/fromDataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(). Internally, timestamps with local time zone are always represented in the UTC time zone. However, when converting to data types that don't include a time zone (e.g. TIMESTAMP, TIME, or simply STRING), the session time zone is used during conversion.Example:
TableConfig config = tEnv.getConfig(); config.setLocalTimeZone(ZoneOffset.ofHours(2)); tEnv.executeSql("CREATE TABLE testTable (id BIGINT, tmstmp TIMESTAMP WITH LOCAL TIME ZONE)"); tEnv.executeSql("INSERT INTO testTable VALUES ((1, '2000-01-01 2:00:00'), (2, TIMESTAMP '2000-01-01 2:00:00'))"); tEnv.executeSql("SELECT * FROM testTable"); // query with local time zone set to UTC+2should produce:
============================= id | tmstmp ============================= 1 | 2000-01-01 2:00:00' 2 | 2000-01-01 2:00:00'If we change the local time zone and query the same table:
config.setLocalTimeZone(ZoneOffset.ofHours(0)); tEnv.executeSql("SELECT * FROM testTable"); // query with local time zone set to UTC+0we should get:
============================= id | tmstmp ============================= 1 | 2000-01-01 0:00:00' 2 | 2000-01-01 0:00:00'- See Also:
-
LocalZonedTimestampType
-
getPlannerConfig
Returns the current configuration of Planner for Table API and SQL queries. -
setPlannerConfig
Sets the configuration of Planner for Table API and SQL queries. Changing the configuration has no effect after the first query has been defined. -
getMaxGeneratedCodeLength
Returns the current threshold where generated code will be split into sub-function calls. Java has a maximum method length of 64 KB. This setting allows for finer granularity if necessary.Default value is 4000 instead of 64KB as by default JIT refuses to work on methods with more than 8K byte code.
-
setMaxGeneratedCodeLength
Sets current threshold where generated code will be split into sub-function calls. Java has a maximum method length of 64 KB. This setting allows for finer granularity if necessary.Default value is 4000 instead of 64KB as by default JIT refuses to work on methods with more than 8K byte code.
-
setIdleStateRetention
Specifies a retention time interval for how long idle state, i.e., state which was not updated, will be retained. State will never be cleared until it was idle for less than the retention time and will be cleared on a best effort basis after the retention time.When new data arrives for previously cleaned-up state, the new data will be handled as if it was the first data. This can result in previous results being overwritten.
Set to 0 (zero) to never clean-up the state.
- Parameters:
duration- The retention time interval for which idle state is retained. Set to 0 (zero) to never clean-up the state.- See Also:
-
StateTtlConfig
-
getIdleStateRetention
- Returns:
- The duration until state which was not updated will be retained.
-
addJobParameter
Sets a custom user parameter that can be accessed viaFunctionContext.getJobParameter(String, String).This will add an entry to the current value of
PipelineOptions.GLOBAL_JOB_PARAMETERS.It is also possible to set multiple parameters at once, which will override any previously set parameters:
Map<String, String> params = ... TableConfig config = tEnv.getConfig(); config.set(PipelineOptions.GLOBAL_JOB_PARAMETERS, params); -
setRootConfiguration
@Internal public void setRootConfiguration(org.apache.flink.configuration.ReadableConfig rootConfiguration) Sets the given configuration asrootConfiguration, which contains any configuration set in the execution context. See the docs ofTableConfigfor more information.- Parameters:
rootConfiguration- root configuration to be set
-
getDefault
-