@PublicEvolving public class TableConfig extends Object
TableEnvironment
session to adjust Table & SQL API
programs.
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()
. Currently, key-value options are only supported for the Blink
planner. Users can configure also underlying execution parameters via this object. E.g.
tEnv.getConfig().addConfiguration(
new Configuration()
.set(CoreOptions.DEFAULT_PARALLELISM, 128)
.set(PipelineOptions.AUTO_WATERMARK_INTERVAL, Duration.ofMillis(800))
.set(ExecutionCheckpointingOptions.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.
ExecutionConfigOptions
,
OptimizerConfigOptions
Constructor and Description |
---|
TableConfig() |
Modifier and Type | Method and Description |
---|---|
void |
addConfiguration(Configuration configuration)
Adds the given key-value configuration to the underlying configuration.
|
void |
addJobParameter(String key,
String value)
Sets a custom user parameter that can be accessed via
FunctionContext.getJobParameter(String, String) . |
Configuration |
getConfiguration()
Gives direct access to the underlying key-value map for advanced configuration.
|
MathContext |
getDecimalContext()
Returns the default context for decimal division calculation.
|
static TableConfig |
getDefault() |
java.time.ZoneId |
getLocalTimeZone()
Returns the current session time zone id.
|
Integer |
getMaxGeneratedCodeLength()
Returns the current threshold where generated code will be split into sub-function calls.
|
long |
getMaxIdleStateRetentionTime() |
long |
getMinIdleStateRetentionTime() |
Boolean |
getNullCheck()
Returns the NULL check.
|
PlannerConfig |
getPlannerConfig()
Returns the current configuration of Planner for Table API and SQL queries.
|
SqlDialect |
getSqlDialect()
Returns the current SQL dialect.
|
void |
setDecimalContext(MathContext decimalContext)
Sets the default context for decimal division calculation.
|
void |
setIdleStateRetentionTime(Time minTime,
Time maxTime)
Specifies a minimum and a maximum time interval for how long idle state, i.e., state which
was not updated, will be retained.
|
void |
setLocalTimeZone(java.time.ZoneId zoneId)
Sets the current session time zone id.
|
void |
setMaxGeneratedCodeLength(Integer maxGeneratedCodeLength)
Returns the current threshold where generated code will be split into sub-function calls.
|
void |
setNullCheck(Boolean nullCheck)
Sets the NULL check.
|
void |
setPlannerConfig(PlannerConfig plannerConfig)
Sets the configuration of Planner for Table API and SQL queries.
|
void |
setSqlDialect(SqlDialect sqlDialect)
Sets the current SQL dialect to parse a SQL query.
|
public Configuration getConfiguration()
public void addConfiguration(Configuration configuration)
configuration
- key-value configuration to be addedpublic SqlDialect getSqlDialect()
public void setSqlDialect(SqlDialect sqlDialect)
public java.time.ZoneId getLocalTimeZone()
TIMESTAMP
WITH LOCAL TIME ZONE
. See setLocalTimeZone(ZoneId)
for more details.LocalZonedTimestampType
public void setLocalTimeZone(java.time.ZoneId zoneId)
DataTypes.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:
TableEnvironment tEnv = ...
TableConfig config = tEnv.getConfig
config.setLocalTimeZone(ZoneOffset.ofHours(2));
tEnv("CREATE TABLE testTable (id BIGINT, tmstmp TIMESTAMP WITH LOCAL TIME ZONE)");
tEnv("INSERT INTO testTable VALUES ((1, '2000-01-01 2:00:00'), (2, TIMESTAMP '2000-01-01 2:00:00'))");
tEnv("SELECT * FROM testTable"); // query with local time zone set to UTC+2
should 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("SELECT * FROM testTable"); // query with local time zone set to UTC+0
we should get:
============================= id | tmstmp ============================= 1 | 2000-01-01 0:00:00' 2 | 2000-01-01 0:00:00'
LocalZonedTimestampType
public Boolean getNullCheck()
public void setNullCheck(Boolean nullCheck)
public PlannerConfig getPlannerConfig()
public void setPlannerConfig(PlannerConfig plannerConfig)
public MathContext getDecimalContext()
MathContext.DECIMAL128
by default.public void setDecimalContext(MathContext decimalContext)
MathContext.DECIMAL128
by default.public Integer getMaxGeneratedCodeLength()
public void setMaxGeneratedCodeLength(Integer maxGeneratedCodeLength)
public void setIdleStateRetentionTime(Time minTime, Time maxTime)
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.
NOTE: Cleaning up state requires additional bookkeeping which becomes less expensive for larger differences of minTime and maxTime. The difference between minTime and maxTime must be at least 5 minutes.
minTime
- The minimum time interval for which idle state is retained. Set to 0 (zero) to
never clean-up the state.maxTime
- The maximum time interval for which idle state is retained. Must be at least 5
minutes greater than minTime. Set to 0 (zero) to never clean-up the state.public long getMinIdleStateRetentionTime()
public long getMaxIdleStateRetentionTime()
@Experimental public void addJobParameter(String key, String value)
FunctionContext.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.getConfiguration().set(PipelineOptions.GLOBAL_JOB_PARAMETERS, params);
public static TableConfig getDefault()
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.