This is an automated email from the ASF dual-hosted git repository.

fhueske 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 cebcd7074a6 [FLINK-39821][table] Make REGEXP_REPLACE return type 
nullable (#28293)
cebcd7074a6 is described below

commit cebcd7074a6c583206125c2c1cfda7151a5e536e
Author: Ramin Gharib <[email protected]>
AuthorDate: Wed Jun 3 11:33:44 2026 +0200

    [FLINK-39821][table] Make REGEXP_REPLACE return type nullable (#28293)
    
    REGEXP_REPLACE used nullableIfArgs, so the planner inferred a NOT NULL 
output when all three arguments were NOT NULL. The runtime returns null for a 
non-literal regex that fails to compile, since a column reference or CONCAT 
result is only validated at runtime, not at planning time. That let a null 
value flow through a column the planner believed was non-null.
    
    Switches the output type strategy to explicit(STRING().nullable()), 
matching REGEXP_EXTRACT which is nullable for the same reason. REGEXP keeps 
nullableIfArgs because its runtime returns false, not null, on an invalid 
pattern.
    
    Adds a RegexpFunctionsITCase case with NOT NULL arguments and a non-literal 
invalid regex, asserting the output type stays nullable and the value is null.
---
 .../flink/table/functions/BuiltInFunctionDefinitions.java    |  2 +-
 .../flink/table/planner/functions/RegexpFunctionsITCase.java | 12 +++++++++++-
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
index 7025a51904f..1d842767252 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
@@ -1726,7 +1726,7 @@ public final class BuiltInFunctionDefinitions {
                     .name("REGEXP_REPLACE")
                     .kind(SCALAR)
                     
.inputTypeStrategy(SpecificInputTypeStrategies.REGEXP_REPLACE)
-                    
.outputTypeStrategy(nullableIfArgs(explicit(DataTypes.STRING())))
+                    
.outputTypeStrategy(explicit(DataTypes.STRING().nullable()))
                     .runtimeProvided()
                     .build();
 
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
index 5d920617512..97ab2568617 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/RegexpFunctionsITCase.java
@@ -433,7 +433,17 @@ class RegexpFunctionsITCase extends 
BuiltInFunctionTestBase {
                                 "Invalid regular expression for 
REGEXP_REPLACE:")
                         .testSqlValidationError(
                                 "REGEXP_REPLACE(f0, '(', 'X')",
-                                "Invalid regular expression for 
REGEXP_REPLACE:"));
+                                "Invalid regular expression for 
REGEXP_REPLACE:"),
+                TestSetSpec.forFunction(
+                                BuiltInFunctionDefinitions.REGEXP_REPLACE,
+                                "Output stays nullable for non-null arguments")
+                        .onFieldsWithData("foobar", "(")
+                        .andDataTypes(DataTypes.STRING().notNull(), 
DataTypes.STRING().notNull())
+                        .testResult(
+                                $("f0").regexpReplace($("f1"), "X"),
+                                "REGEXP_REPLACE(f0, f1, 'X')",
+                                null,
+                                DataTypes.STRING().nullable()));
     }
 
     private Stream<TestSetSpec> regexpSubstrTestCases() {

Reply via email to