This is an automated email from the ASF dual-hosted git repository. twalthr pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push: new 50ffe42 [FLINK-25516][table-api-java] Add catalog object compile/restore options 50ffe42 is described below commit 50ffe425bd2070cd4a7f976e2360b8742cc83d8a Author: Timo Walther <twal...@apache.org> AuthorDate: Tue Jan 4 11:29:17 2022 +0100 [FLINK-25516][table-api-java] Add catalog object compile/restore options This closes #18262. --- .../generated/table_config_configuration.html | 12 +++ .../flink/table/api/config/TableConfigOptions.java | 120 +++++++++++++++++++++ 2 files changed, 132 insertions(+) diff --git a/docs/layouts/shortcodes/generated/table_config_configuration.html b/docs/layouts/shortcodes/generated/table_config_configuration.html index ca94bf0..c5e2be9 100644 --- a/docs/layouts/shortcodes/generated/table_config_configuration.html +++ b/docs/layouts/shortcodes/generated/table_config_configuration.html @@ -33,6 +33,18 @@ <td>The local time zone defines current session time zone id. It is used when converting to/from <code>TIMESTAMP WITH LOCAL TIME ZONE</code>. 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. The input of option is either a full name such as "America/Los_Angeles", or [...] </tr> <tr> + <td><h5>table.plan.compile.catalog-objects</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td> + <td style="word-wrap: break-word;">ALL</td> + <td><p>Enum</p></td> + <td>Strategy how to persist catalog objects such as tables, functions, or data types into a plan during compilation. It influences the need for catalog metadata to be present during a restore operation and affects the plan size.<br /><br />Possible values:<ul><li>"ALL": All metadata about catalog tables, functions, or data types will be persisted into the plan during compilation. For catalog tables, this includes the table's identifier, schema, and options. For catalog functi [...] + </tr> + <tr> + <td><h5>table.plan.restore.catalog-objects</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td> + <td style="word-wrap: break-word;">ALL</td> + <td><p>Enum</p></td> + <td>Strategy how to restore catalog objects such as tables, functions, or data types using a given plan and performing catalog lookups if necessary. It influences the need for catalog metadata to bepresent and enables partial enrichment of plan information.<br /><br />Possible values:<ul><li>"ALL": Reads all metadata about catalog tables, functions, or data types that has been persisted in the plan. The strategy performs a catalog lookup by identifier to fill in missing infor [...] + </tr> + <tr> <td><h5>table.sql-dialect</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td> <td style="word-wrap: break-word;">"default"</td> <td>String</td> diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java index 22b5b26..50c5dcb 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java @@ -18,13 +18,18 @@ package org.apache.flink.table.api.config; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.DescribedEnum; +import org.apache.flink.configuration.description.InlineElement; import org.apache.flink.table.api.PlannerType; import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.catalog.Catalog; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.description.TextElement.text; /** * This class holds {@link org.apache.flink.configuration.ConfigOption}s used by table planner. @@ -90,6 +95,36 @@ public class TableConfigOptions { + "the session time zone is used during conversion. The input of option is either a full name " + "such as \"America/Los_Angeles\", or a custom timezone id such as \"GMT-08:00\"."); + // ------------------------------------------------------------------------------------------ + // Options for plan handling + // ------------------------------------------------------------------------------------------ + + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption<CatalogPlanCompilation> PLAN_COMPILE_CATALOG_OBJECTS = + key("table.plan.compile.catalog-objects") + .enumType(CatalogPlanCompilation.class) + .defaultValue(CatalogPlanCompilation.ALL) + .withDescription( + "Strategy how to persist catalog objects such as tables, functions, or data " + + "types into a plan during compilation. It influences the need " + + "for catalog metadata to be present during a restore operation " + + "and affects the plan size."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption<CatalogPlanRestore> PLAN_RESTORE_CATALOG_OBJECTS = + key("table.plan.restore.catalog-objects") + .enumType(CatalogPlanRestore.class) + .defaultValue(CatalogPlanRestore.ALL) + .withDescription( + "Strategy how to restore catalog objects such as tables, functions, or data " + + "types using a given plan and performing catalog lookups if " + + "necessary. It influences the need for catalog metadata to be" + + "present and enables partial enrichment of plan information."); + + // ------------------------------------------------------------------------------------------ + // Options for code generation + // ------------------------------------------------------------------------------------------ + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) public static final ConfigOption<Integer> MAX_LENGTH_GENERATED_CODE = key("table.generated-code.max-length") @@ -108,4 +143,89 @@ public class TableConfigOptions { .defaultValue(10000) .withDescription( "Specifies a threshold where class members of generated code will be grouped into arrays by types."); + + // ------------------------------------------------------------------------------------------ + // Enum option types + // ------------------------------------------------------------------------------------------ + + /** Strategy to compile {@link Catalog} objects into a plan. */ + @PublicEvolving + public enum CatalogPlanCompilation implements DescribedEnum { + ALL( + text( + "All metadata about catalog tables, functions, or data types will " + + "be persisted into the plan during compilation. For catalog tables, " + + "this includes the table's identifier, schema, and options. For " + + "catalog functions, this includes the function's identifier and " + + "class. For catalog data types, this includes the identifier and " + + "entire type structure. " + + "With this strategy, the catalog's metadata doesn't have to be " + + "available anymore during a restore operation.")), + SCHEMA( + text( + "In addition to an identifier, schema information about catalog tables, " + + "functions, or data types will be persisted into the plan during " + + "compilation. A schema allows for detecting incompatible changes " + + "in the catalog during a plan restore operation. However, all " + + "other metadata will still be retrieved from the catalog.")), + + IDENTIFIER( + text( + "Only the identifier of catalog tables, functions, or data types will be " + + "persisted into the plan during compilation. All metadata will " + + "be retrieved from the catalog during a restore operation. With " + + "this strategy, plans become less verbose.")); + + private final InlineElement description; + + CatalogPlanCompilation(InlineElement description) { + this.description = description; + } + + @Internal + @Override + public InlineElement getDescription() { + return description; + } + } + + /** Strategy to restore {@link Catalog} objects using the plan and lookups if necessary. */ + @PublicEvolving + public enum CatalogPlanRestore implements DescribedEnum { + ALL( + text( + "Reads all metadata about catalog tables, functions, or data types that has " + + "been persisted in the plan. The strategy performs a catalog " + + "lookup by identifier to fill in missing information or enrich " + + "mutable options. If the original object is not available in the " + + "catalog anymore, pipelines can still be restored if all information " + + "necessary is contained in the plan.")), + ALL_ENFORCED( + text( + "Requires that all metadata about catalog tables, functions, or data types " + + "that has been persisted in the plan. The strategy will neither " + + "perform a catalog lookup by identifier nor enrich mutable " + + "options with catalog information. A restore will fail if not all " + + "information necessary is contained in the plan.")), + + IDENTIFIER( + text( + "Uses only the identifier of catalog tables, functions, or data types and " + + "always performs a catalog lookup. A restore will fail if the " + + "original object is not available in the catalog anymore. " + + "Additional metadata that might be contained in the plan will " + + "be ignored.")); + + private final InlineElement description; + + CatalogPlanRestore(InlineElement description) { + this.description = description; + } + + @Internal + @Override + public InlineElement getDescription() { + return description; + } + } }