Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

Current state: "Under Discussion"

Discussion thread: here (<- link to https://mail-archives.apache.org/mod_mbox/flink-dev/)

JIRA: here (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)

...

Page properties


Discussion thread
Vote thread
JIRA
Release


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

The goal of those changes is to improve the TableConfig so it possible to configure current planner settings through ConfigOptions.  It also aims to improve the configuration user experience.

Public Interfaces

New ConfigOptions

keysettingtypeClass to place it in
table.exec.min-idle-state-retentionTableConfig#setIdleStateRetentionTimeDurationExecutionConfigOptions
table.exec.max-idle-state-retentionTableConfig#setIdleStateRetentionTimeDurationExecutionConfigOptions
table.planner.sql-dialectTableConfig#setSqlDialectEnum<SqlDialect>PlannerConfigOptions
table.planner.time-zone-idTableConfig#setLocalTimeZoneString (ZoneId.of())PlannerConfigOptionstable.planner.null-checkTableConfig#setNullCheckBooleanPlannerConfigOptions
table.planner.max-generated-codeTableConfig#setMaxGeneratedCodeLengthIntegerPlannerConfigOptions
table.planner.math-context.precisionTableConfig#setDecimalContextInteger (precision of MatchContextMathContext)PlannerConfigOptions
table.planner.math-context.rounding-modeTableConfig#setDecimalContextEnum<MatchContext.RoundingMode>PlannerConfigOptions

Changes to TableConfig

  • change the class to be an interface.
  • Remove getConfiguration method
  • Remove getDefault method
  • Make TableConfig extend from ReadableConfig & WritableConfig
  • Deprecate {{setNullCheck}}


Code Block
@PublicEvolving
public interface TableConfig extends ReadableConfig, WritableConfig {

  /*
       Old setters/getters that will be stored in a Configuration in the TableConfigImpl.
   */

.....
	/**
	 * Returns the current SQL dialect.
	 */
	SqlDialect getSqlDialect();

	/**
	 * Sets the current SQL dialect to parse a SQL query. Flink's SQL behavior by default.
	 */
	void setSqlDialect(SqlDialect sqlDialect);
......


  /*
		Methods inherited from ReadableConfig/WritableConfig

		<T> TableConfig set(ConfigOption<T> option, T value);
		
		<T> Optional<T> getOptional(ConfigOption<T> option);

		<T> T get(ConfigOption<T> option);

   */

   /*
      Methods to be dropped:

   	  public static TableConfig getDefault();
      public Configuration getConfiguration();

    */

    

}


Changes to (Stream)TableEnvironment

  • drop deprecated create()  methods that create TableEnvironment with a given TableConfig 

Proposed Changes

First of the suggestion is to change the TableConfig to be an interface. This makes future changes to the TableConfig easier. It also moves the control over the lifecycle of the TableConfig to the {{TableEnvironment}}. Along with this change we will remove deprecated methods that create StreamTableEnvironment with a TableConfig

...

Code Block
tEnv.getConfig()
	.set(CoreOptions.DEFAULT_PARALLELISM, 128)
	.set(PipelineOptions.AUTO_WATERMARK_INTERVAL, Duration.ofMillis(800))
	.set(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(30));


Compatibility, Deprecation, and Migration Plan

This flip suggests two breaking changes:

...