[FLINK-38839][runtime] Support semicolon delimiter for table-options#4199
[FLINK-38839][runtime] Support semicolon delimiter for table-options#4199lvyanquan merged 5 commits intoapache:masterfrom
Conversation
.../src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java
Outdated
Show resolved
Hide resolved
|
Thanks for the suggestion @leonardBang! I've updated the PR to support custom delimiters as you recommended. Changes made:
Example usage: transform:
- source-table: mydb.mytable
table-options: sequence.field=gxsj,jjsj;file-index.bloom-filter.columns=jjdbh
table-options.delimiter: ";"This approach is more flexible and powerful as users can now choose any delimiter that suits their needs. |
There was a problem hiding this comment.
Pull request overview
This PR extends transform rule support for table-options by introducing a configurable key/value-pair delimiter (documented with semicolon as the primary use case) so option values can safely contain commas, while preserving comma-delimited behavior for existing configs.
Changes:
- Add
table-options.delimiterto transform definitions (CLI YAML parsing → composer definition → operator builders/runtime rule propagation). - Update
SchemaMetadataTransformparsing to support custom delimiters and split on the first=only. - Add documentation and unit tests covering delimiter behavior and comma/equals in option values.
Reviewed changes
Copilot reviewed 11 out of 11 changed files in this pull request and generated 6 comments.
Show a summary per file
| File | Description |
|---|---|
| flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransformTest.java | Adds unit tests for table-options parsing with different delimiters and values containing , / =. |
| flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java | Carries tableOptionsDelimiter through the runtime rule model. |
| flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java | Implements delimiter-based parsing and split("=", 2) for options. |
| flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java | Adds delimiter-aware overload for constructing transform rules. |
| flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java | Passes delimiter into SchemaMetadataTransform. |
| flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorBuilder.java | Adds delimiter-aware overload for constructing transform rules. |
| flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java | Wires delimiter from TransformDef into operator builders. |
| flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/TransformDef.java | Adds tableOptionsDelimiter to the transform definition model. |
| flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java | Parses table-options.delimiter from YAML into TransformDef. |
| docs/content/docs/core-concept/transform.md | Documents table-options.delimiter and provides semicolon example. |
| docs/content.zh/docs/core-concept/transform.md | Same documentation update for the Chinese docs. |
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
...main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorBuilder.java
Show resolved
Hide resolved
.../src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java
Outdated
Show resolved
Hide resolved
.../src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java
Show resolved
Hide resolved
...dc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java
Show resolved
Hide resolved
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/TransformDef.java
Outdated
Show resolved
Hide resolved
.../main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java
Show resolved
Hide resolved
5870b30 to
34ccd84
Compare
...main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorBuilder.java
Outdated
Show resolved
Hide resolved
.../src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java
Outdated
Show resolved
Hide resolved
34ccd84 to
e130dc7
Compare
Co-authored-by: yuxiqian <34335406+yuxiqian@users.noreply.github.com>
yuxiqian
left a comment
There was a problem hiding this comment.
Looks good, waiting for CI...
…pache#4199) Co-authored-by: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Co-authored-by: Leonard Xu <xbjtdcq@gmail.com>
Summary
Allow
table-optionsin transform rules to use semicolon;as the key/value pair delimiter so option values can safely contain commas (e.g.sequence.field=gxsj,jjsj). Keep comma delimiter for backward compatibility.Motivation
Some downstream table options are multi-value and use comma inside the value. The existing
key1=value1,key2=value2syntax makes those options impossible to express reliably.Changes
SchemaMetadataTransformparsestable-optionspairs using:;when present in the string, otherwise,split("=", 2)to avoid breaking values containing=Behavior
table-options: key1=value1,key2=value2table-options: sequence.field=gxsj,jjsj;file-index.range-bitmap.columns=jjsj;file-index.bloom-filter.columns=jjdbh;(do not mix,and;in the same string).