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

gyfora 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 a575084d884 [FLINK-38969][SQL] Extend json_string to convert variant 
type
a575084d884 is described below

commit a575084d884d04cbedab66500af44343b7f6eb9f
Author: Swapna Marru <[email protected]>
AuthorDate: Wed Feb 11 23:02:34 2026 -0800

    [FLINK-38969][SQL] Extend json_string to convert variant type
---
 docs/data/sql_functions.yml                                   |  5 +++++
 docs/data/sql_functions_zh.yml                                |  6 ++++++
 .../inference/strategies/SpecificInputTypeStrategies.java     |  3 ++-
 .../flink/table/planner/codegen/JsonGenerateUtils.scala       |  3 +++
 .../flink/table/planner/functions/JsonFunctionsITCase.java    | 11 +++++++++--
 5 files changed, 25 insertions(+), 3 deletions(-)

diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml
index 0bf149795e9..0be8e6c9ca2 100644
--- a/docs/data/sql_functions.yml
+++ b/docs/data/sql_functions.yml
@@ -942,6 +942,8 @@ json:
       JSON_STRING('Hello, World!')
       -- '[1,2]'
       JSON_STRING(ARRAY[1, 2])
+      -- '{"k1":"v1"}'
+      JSON_STRING(PARSE_JSON('{"k1":"v1"}'))
       ```
   - sql: JSON_VALUE(jsonValue, path [RETURNING <dataType>] [ { NULL | ERROR | 
DEFAULT <defaultExpr> } ON EMPTY ] [ { NULL | ERROR | DEFAULT <defaultExpr> } 
ON ERROR ])
     table: STRING.jsonValue(STRING path [, returnType, onEmpty, 
defaultOnEmpty, onError, defaultOnError])
@@ -1167,6 +1169,9 @@ variant:
       parser will keep the last occurrence of all fields with the same key, 
otherwise when 
       `allowDuplicateKeys` is false it will throw an error. The default value 
of 
       `allowDuplicateKeys` is false.
+  - sql: JSON_STRING(variant)
+    description: |
+      Generate a json string from a Variant object.
   - sql: variant '[' INT ']'
     table: VARIANT.at(INT)
     description: |
diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml
index 9d2541babe6..1c70ff12d11 100644
--- a/docs/data/sql_functions_zh.yml
+++ b/docs/data/sql_functions_zh.yml
@@ -1067,6 +1067,8 @@ json:
       JSON_STRING('Hello, World!')
       -- '[1,2]'
       JSON_STRING(ARRAY[1, 2])
+      -- '{"k1":"v1"}'
+      JSON_STRING(PARSE_JSON('{"k1":"v1"}'))
       ```
   - sql: JSON_VALUE(jsonValue, path [RETURNING <dataType>] [ { NULL | ERROR | 
DEFAULT <defaultExpr> } ON EMPTY ] [ { NULL | ERROR | DEFAULT <defaultExpr> } 
ON ERROR ])
     table: STRING.jsonValue(STRING path [, returnType, onEmpty, 
defaultOnEmpty, onError, defaultOnError])
@@ -1250,6 +1252,10 @@ variant:
       同键的字段,否则当 allowDuplicateKeys 为 false 时,它会抛出一个错误。默认情况下,
       allowDuplicateKeys 的值为 false。
 
+  - sql: JSON_STRING(variant)
+    description: |
+      Generate a json string from a Variant object.
+
   - sql: variant '[' INT ']'
     table: VARIANT.at(INT)
     description: |
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
index c0429b7a46e..9206985aa13 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/SpecificInputTypeStrategies.java
@@ -89,7 +89,8 @@ public final class SpecificInputTypeStrategies {
                     logical(LogicalTypeFamily.NUMERIC),
                     logical(LogicalTypeRoot.STRUCTURED_TYPE),
                     logical(LogicalTypeRoot.DISTINCT_TYPE),
-                    logical(LogicalTypeRoot.BOOLEAN));
+                    logical(LogicalTypeRoot.BOOLEAN),
+                    logical(LogicalTypeRoot.VARIANT));
 
     /** See {@link JsonQueryOnErrorEmptyArgumentTypeStrategy}. */
     public static final ArgumentTypeStrategy 
JSON_QUERY_ON_EMPTY_ERROR_BEHAVIOUR =
diff --git 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
index 823837d2f5e..0fec3d1b23c 100644
--- 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
+++ 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
@@ -119,6 +119,9 @@ object JsonGenerateUtils {
       case DISTINCT_TYPE =>
         createNodeTerm(ctx, valueTerm, 
valueType.asInstanceOf[DistinctType].getSourceType)
 
+      case VARIANT =>
+        s"$nodeFactoryTerm.rawValueNode(new 
${typeTerm(classOf[RawValue])}($valueTerm.toJson()))"
+
       case _ =>
         throw new CodeGenException(
           s"Type '$valueType' is not scalar or cannot be converted into JSON.")
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
index e26401951c4..5905da188cf 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
@@ -621,7 +621,8 @@ class JsonFunctionsITCase extends BuiltInFunctionTestBase {
                                 multisetData,
                                 "Test".getBytes(StandardCharsets.UTF_8),
                                 "Test".getBytes(StandardCharsets.UTF_8),
-                                Row.of(Collections.singletonList(Row.of(1, 
2))))
+                                Row.of(Collections.singletonList(Row.of(1, 
2))),
+                                "{\"key\":\"value\"}")
                         .andDataTypes(
                                 STRING().notNull(),
                                 BOOLEAN().notNull(),
@@ -636,7 +637,8 @@ class JsonFunctionsITCase extends BuiltInFunctionTestBase {
                                 MAP(STRING(), INT()).notNull(),
                                 BINARY(4).notNull(),
                                 VARBINARY(4).notNull(),
-                                ROW(ARRAY(ROW(INT(), INT()))).notNull())
+                                ROW(ARRAY(ROW(INT(), INT()))).notNull(),
+                                STRING().notNull())
                         .testResult(
                                 jsonString($("f0")), "JSON_STRING(f0)", 
"\"V\"", STRING().notNull())
                         .testResult(
@@ -690,6 +692,11 @@ class JsonFunctionsITCase extends BuiltInFunctionTestBase {
                                 jsonString($("f13")),
                                 "JSON_STRING(f13)",
                                 "{\"f0\":[{\"f0\":1,\"f1\":2}]}",
+                                STRING().notNull())
+                        .testResult(
+                                jsonString(call("PARSE_JSON", $("f14"))),
+                                
"JSON_STRING(PARSE_JSON('{\"key\":\"value\"}'))",
+                                "{\"key\":\"value\"}",
                                 STRING().notNull()));
     }
 

Reply via email to