[FLINK-39663][table] Support ALTER MATERIALIZED TABLE ... RESET (...) conversion#28165
Conversation
… conversion
The parser already accepted `ALTER MATERIALIZED TABLE t RESET ('k')`, but the planner had no converter registered for `SqlAlterMaterializedTableReset`. As a result, the statement was rejected at plan time with `TableException: Unsupported query`.
Add `SqlAlterMaterializedTableResetConverter` and register it in `SqlNodeConverters#registerMaterializedTableConverters`. The converter emits an `AlterMaterializedTableChangeOperation` with one `TableChange.reset(key)` per key. Merging is handled by `MaterializedTableChangeHandler.resetTableOption`, which is a no-op for keys not currently set. Empty key lists and resetting the `connector` key are rejected, matching the non-materialized-table RESET path.
Planner-level tests cover success (with order-preserving summary string), empty key list, the `connector` key, unknown table, and applying RESET to a regular table. A gateway IT test exercises the end-to-end flow in FULL refresh mode where the change is applied directly to the catalog. The user-facing docs are updated for the new RESET clause.
| final SqlNodeList propertyKeyList = sqlAlterTable.getPropertyKeyList(); | ||
| if (propertyKeyList.getList().isEmpty()) { | ||
| throw new ValidationException( | ||
| EX_MSG_PREFIX + "ALTER MATERIALIZED TABLE RESET does not support empty key."); | ||
| } | ||
| final Set<String> resetKeys = new LinkedHashSet<>(propertyKeyList.size()); | ||
| for (SqlNode key : propertyKeyList) { | ||
| resetKeys.add(SqlParseUtils.extractString((SqlLiteral) key)); | ||
| } |
There was a problem hiding this comment.
use SqlParseUtils.extractList instead
| } | ||
|
|
||
| @Test | ||
| void testAlterMaterializedTableResetWithEmptyKey() { |
There was a problem hiding this comment.
can we parameterized similar tests or probably just use already existing parameterized tests?
There was a problem hiding this comment.
Thanks! Addressed!
| final List<String> resetKeys = | ||
| SqlParseUtils.extractList( | ||
| propertyKeyList, key -> SqlParseUtils.extractString((SqlLiteral) key)); | ||
| if (resetKeys.contains(FactoryUtil.CONNECTOR.key())) { |
There was a problem hiding this comment.
why do we hard fail if there is connector option and do not fail hard if there is unknown option?
There was a problem hiding this comment.
Good point! I was looking into the code of SqlAlterTableResetConverter. There we have similar logic:
https://github.com/raminqaf/flink/blob/b7780d324def665c6ca406e8d83393594eb6278b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableResetConverter.java#L44-L44
There was a problem hiding this comment.
what happen if there are 5 valid options and 5 unknown options?
will user be able to see which are reset and which are unknown?
There was a problem hiding this comment.
If we introduce a hard fail for non-exiting option we should consider to do the same for Tables too
| final SqlNodeList propertyKeyList = sqlAlterTable.getPropertyKeyList(); | ||
| if (propertyKeyList.getList().isEmpty()) { | ||
| throw new ValidationException( | ||
| EX_MSG_PREFIX + "ALTER MATERIALIZED TABLE RESET does not support empty key."); | ||
| } | ||
| final List<String> resetKeys = | ||
| SqlParseUtils.extractList( | ||
| propertyKeyList, key -> SqlParseUtils.extractString((SqlLiteral) key)); |
There was a problem hiding this comment.
I guess first we can extract, then check if it is empty in order to avoid working with SqlNodeList here
| throw new ValidationException( | ||
| EX_MSG_PREFIX + "ALTER MATERIALIZED TABLE RESET does not support empty key."); | ||
| } | ||
| final List<String> resetKeys = |
There was a problem hiding this comment.
can we use Set?
- later anyway there is contains check
- will eliminate duplicates
Spell out the behavior for unknown keys (silently ignored), duplicate keys (de-duplicated), the empty key list (rejected), and the reserved `connector` key (rejected). Add corresponding examples.
What is the purpose of the change
The parser already accepted
ALTER MATERIALIZED TABLE t RESET ('k'), but the planner had no converter registered forSqlAlterMaterializedTableReset. As a result, the statement was rejected at plan time withTableException: Unsupported query.Add
SqlAlterMaterializedTableResetConverterand register it inSqlNodeConverters#registerMaterializedTableConverters. The converter emits anAlterMaterializedTableChangeOperationwith oneTableChange.reset(key)per key. Merging is handled byMaterializedTableChangeHandler.resetTableOption, which is a no-op for keys not currently set. Empty key lists and resetting theconnectorkey are rejected, matching the non-materialized-table RESET path.Planner-level tests cover success (with order-preserving summary string), empty key list, the
connectorkey, unknown table, and applying RESET to a regular table. A gateway IT test exercises the end-to-end flow in FULL refresh mode where the change is applied directly to the catalog. The user-facing docs are updated for the new RESET clause.Brief change log
SqlAlterMaterializedTableResetConverterSqlNodeConverters#registerMaterializedTableConvertersVerifying this change
SqlMaterializedTableNodeToOperationConverterTest.javaMaterializedTableStatementITCase.javaDoes this pull request potentially affect one of the following parts:
@Public(Evolving): (no)Documentation
Was generative AI tooling used to co-author this PR?
Generated-by: Opus 4.7