Copilot commented on code in PR #4199:
URL: https://github.com/apache/flink-cdc/pull/4199#discussion_r2959794185


##########
flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperatorBuilder.java:
##########
@@ -44,6 +44,28 @@ public PostTransformOperatorBuilder addTransform(
             String tableOptions,
             String postTransformConverter,
             SupportedMetadataColumn[] supportedMetadataColumns) {
+        return addTransform(
+                tableInclusions,
+                projection,
+                filter,
+                primaryKey,
+                partitionKey,
+                tableOptions,
+                null,
+                postTransformConverter,
+                supportedMetadataColumns);
+    }
+
+    public PostTransformOperatorBuilder addTransform(
+            String tableInclusions,
+            @Nullable String projection,
+            @Nullable String filter,
+            String primaryKey,
+            String partitionKey,
+            String tableOptions,
+            String tableOptionsDelimiter,

Review Comment:
   `tableOptionsDelimiter` is passed as `null` from the overload above, but 
this overload declares it as a non-null `String`. This makes the method 
contract misleading and inconsistent with the use of `@Nullable` elsewhere. 
Mark it as `@Nullable` or default it before constructing the `TransformRule`.
   



##########
flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformRule.java:
##########
@@ -44,6 +45,7 @@ public TransformRule(
             String primaryKey,
             String partitionKey,
             String tableOption,
+            String tableOptionsDelimiter,
             @Nullable String postTransformConverter,
             SupportedMetadataColumn[] supportedMetadataColumns) {

Review Comment:
   `tableOptionsDelimiter` is treated as optional (builders pass `null`), but 
the constructor parameter is declared as non-null `String`. This makes the API 
contract misleading and can trip nullness checks. Either normalize it to a 
non-null default (e.g., `,`) inside the constructor, or mark it (and the 
corresponding field/getter) as `@Nullable`.



##########
flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java:
##########
@@ -58,13 +61,19 @@ public SchemaMetadataTransform(
             partitionKeys = Arrays.asList(partitionKeyArr);
         }
         if (!StringUtils.isNullOrWhitespaceOnly(tableOptionString)) {
-            for (String tableOption : tableOptionString.split(",")) {
-                String[] kv = tableOption.split("=");
+            // Use custom delimiter if provided, otherwise default to comma 
for backward
+            // compatibility.
+            String delimiter =
+                    StringUtils.isNullOrWhitespaceOnly(tableOptionsDelimiter)
+                            ? ","

Review Comment:
   The PR description says the delimiter should be chosen automatically (use 
`;` when it appears in `table-options`, otherwise `,`). Currently `;` is only 
used when an explicit `tableOptionsDelimiter` is provided, so `table-options: 
k1=v1;k2=v2` without `table-options.delimiter` will be parsed as a single 
option. Consider adding auto-detection when `tableOptionsDelimiter` is 
null/blank (while still allowing an explicit override).



##########
flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java:
##########
@@ -57,6 +58,28 @@ public PreTransformOperatorBuilder addTransform(
             String tableOption,
             @Nullable String postTransformConverter,
             SupportedMetadataColumn[] supportedMetadataColumns) {
+        return addTransform(
+                tableInclusions,
+                projection,
+                filter,
+                primaryKey,
+                partitionKey,
+                tableOption,
+                null,
+                postTransformConverter,
+                supportedMetadataColumns);
+    }
+
+    public PreTransformOperatorBuilder addTransform(
+            String tableInclusions,
+            @Nullable String projection,
+            @Nullable String filter,
+            String primaryKey,
+            String partitionKey,
+            String tableOption,
+            String tableOptionsDelimiter,

Review Comment:
   `tableOptionsDelimiter` is passed as `null` from the overload above, but 
this overload declares it as a non-null `String`. To keep the API contract 
accurate (and consistent with other optional params), mark 
`tableOptionsDelimiter` as `@Nullable` or normalize it to a default value 
before calling this overload.
   



##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/TransformDef.java:
##########
@@ -36,6 +36,8 @@
  *       by `,`. Optional for the definition.
  *   <li>tableOptions: a string for table options for matching input table 
IDs, options are
  *       seperated by `,`, key and value are seperated by `=`. Optional for 
the definition.
+ *   <li>tableOptionsDelimiter: a string for delimiter of table options, 
default is `,`. Optional

Review Comment:
   The TransformDef javadoc still says table options are "seperated by `,`" and 
contains the typo "seperated". Since this PR introduces 
`tableOptionsDelimiter`, the comment should be updated to reflect the 
configurable delimiter and fix the spelling to "separated".



##########
flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/SchemaMetadataTransform.java:
##########
@@ -58,13 +61,19 @@ public SchemaMetadataTransform(
             partitionKeys = Arrays.asList(partitionKeyArr);
         }
         if (!StringUtils.isNullOrWhitespaceOnly(tableOptionString)) {
-            for (String tableOption : tableOptionString.split(",")) {
-                String[] kv = tableOption.split("=");
+            // Use custom delimiter if provided, otherwise default to comma 
for backward
+            // compatibility.
+            String delimiter =
+                    StringUtils.isNullOrWhitespaceOnly(tableOptionsDelimiter)
+                            ? ","
+                            : tableOptionsDelimiter;
+            for (String tableOption : tableOptionString.split(delimiter)) {

Review Comment:
   `tableOptionString.split(delimiter)` treats `delimiter` as a regex. This 
breaks literal delimiters like `|` and `$` (and can throw 
`PatternSyntaxException`), which your new tests/docs suggest should work. Split 
using a quoted/literal delimiter (e.g., `Pattern.quote(delimiter)` or a 
precompiled `Pattern`) so custom delimiters are interpreted literally.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to