Status
Current state: "Accepted"
Discussion thread: https://lists.apache.org/thread/1sthbv6q00sq52pp04n2p26d70w4fqj1
JIRA:
-
FLINK-35473Getting issue details...
STATUS
Released: <Flink Version>
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
As Flink moves toward 2.0, we have revisited all table/SQL configurations and identified several improvements to enhance user-friendliness and maintainability. In this FLIP, we aim to refine them.
Public Interfaces and Proposed Changes
Module | Configuration Options | Class | Annotation | Proposed Changes | Explanation |
---|---|---|---|---|---|
flink-table-planner | table async output-mode capacity timeout retry-predicate retry-strategy fixed-delay max-attempts | org.apache.flink.table.planner.hint.LookupJoinHintOptions | Internal |
| All configurations are user-facing and directly exposed via the doc. See https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/queries/hints/#lookup-hint-options |
table.exec.local-hash-agg.adaptive.enabled table.exec.local-hash-agg.adaptive.sampling-threshold table.exec.local-hash-agg.adaptive.distinct-value-rate-threshold | org.apache.flink.table.planner.codegen.agg.batch.HashAggCodeGenerator | Experimental |
| All configurations are prefixed with "table.exec" | |
table.optimizer.incremental-agg-enabled | org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule |
| All configurations are prefixed with "table.optimizer" | ||
table.optimizer.union-all-as-breakpoint-enabled table.optimizer.reuse-optimize-block-with-digest-enabled | org.apache.flink.table.planner.plan.optimize.RelNodeBlock | ||||
table.exec.range-sort.enabled | org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortRule | Experimental | To be deprecated and removed | As quoted from discussion thread
| |
table.optimizer.rows-per-local-agg | org.apache.flink.table.planner.plan.metadata.FlinkRelMdRowCount | ||||
table.optimizer.join.null-filter-threshold | org.apache.flink.table.planner.plan.rules.logical.JoinDeriveNullFilterRule | ||||
table.optimizer.semi-anti-join.build-distinct.ndv-ratio table.optimizer.shuffle-by-partial-key-enabled | org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase | ||||
table.optimizer.smj.remove-sort-enabled | org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalSortMergeJoinRule | ||||
table.optimizer.cnf-nodes-limit | org.apache.flink.table.planner.plan.utils.FlinkRexUtil | ||||
flink-table-api-java | table.optimizer.source.predicate-pushdown-enabled | org.apache.flink.table.api.config.OptimizerConfigOptions | PublicEvolving |
| The reason is discussed in FLIP-377. As quoted on https://lists.apache.org/thread/z9fdknogt5265o0vg3q4swqj5to8rp9x
|
flink-sql-client | sql-client.display.max-column-width | org.apache.flink.table.client.config.SqlClientOptions | N.A. | 1. Mark the class as PublicEvolving; 2. Deprecate and remove "sql-client.display.max-column-width" | The alternative to "sql-client.display.max-column-width" is "table.display.max-column-width" |
N.A. | org.apache.flink.table.client.config.ResultMode | Mark the class as PublicEvolving | Currently, the annotation is missing |
Special Case
The management of the relevant configurations in the org.apache.flink.table.planner.plan.utils.WindowEmitStrategy class hinges on the complete removal of legacy window aggregate in version 2.0. The decision regarding this configuration will be consistently monitored and addressed through - FLINK-29692Getting issue details... STATUS .Configuration Options |
---|
table.exec.emit.early-fire.enabled |
table.exec.emit.early-fire.delay |
table.exec.emit.late-fire.enabled |
table.exec.emit.late-fire.delay |
table.exec.emit.allow-lateness |
Compatibility, Deprecation, and Migration Plan
In Flink1.20, all cited options missing annotations will be marked.
For options to be deprecated:
In Flink 1.20, all options that need to be deprecated will be marked with the Deprecated annotation explicitly and removed in Flink 2.0.
For options to be moved to another module/package:
In Flink 1.20, the options will be deprecated. The new options with the same key will be introduced to the target module/package.
In Flink 2.0, the deprecated options will be removed.
Test Plan
Existing UT/IT can ensure compatibility with old options. New tests will cover the new options.
Rejected Alternatives
N.A.