You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 10 Current »

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)

Released: <Flink Version>

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-zoneTableConfig#setLocalTimeZoneString (ZoneId.of())PlannerConfigOptions
table.planner.max-generated-codeTableConfig#setMaxGeneratedCodeLengthIntegerPlannerConfigOptions
table.planner.math-context.precisionTableConfig#setDecimalContextInteger (precision of MathContext)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}}


@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

We suggest to remove the getConfiguration  method as it exposed too much of internal implementation. We might want to change the underlying way how we persist/store options, which might be hard to do with the current contract of getConfiguration  which gives a read/write access. The write access should happen either through addConfiguration  or TableConfig#set 

Both approaches are easier from user perspective than the current solution  tEnv.getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 128)

For setting a batch of properties (or string properties) users can use addConfiguration  method:

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))
);

// or string - string options

Configuration config = new Configuration();
config.setString("default.parallelism", "128");
config.setString("pipeline.auto-watermark-interval", "800 ms");
config.setString("execution.checkpointing.interval", "30 s");
tEnv.getConfig().addConfiguration(config);

The other option is to set options on TableConfig  explicitly:

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:

  • TableConfig can no longer be instantiated. → This was already discouraged.
  • Remove getConfiguration  method. There are alternatives to this method. Either through addConfiguration  or TableConfig#set 


  • No labels