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

snuyanzin 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 d7206187f52 [FLINK-38796][table] `UnresolvedMetadataColumn` misses 
type during serializing to string
d7206187f52 is described below

commit d7206187f52be9b03569aa68a6c5647f180dc25f
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Tue Dec 9 21:47:06 2025 +0100

    [FLINK-38796][table] `UnresolvedMetadataColumn` misses type during 
serializing to string
---
 flink-python/pyflink/table/tests/test_schema.py                       | 4 ++--
 .../java/org/apache/flink/table/catalog/SchemaResolutionTest.java     | 4 ++--
 .../src/main/java/org/apache/flink/table/api/Schema.java              | 3 +--
 .../operations/SqlMaterializedTableNodeToOperationConverterTest.java  | 2 +-
 4 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/flink-python/pyflink/table/tests/test_schema.py 
b/flink-python/pyflink/table/tests/test_schema.py
index cefac2f3dab..46968c9fad3 100644
--- a/flink-python/pyflink/table/tests/test_schema.py
+++ b/flink-python/pyflink/table/tests/test_schema.py
@@ -51,9 +51,9 @@ class SchemaTest(PyFlinkTestCase):
   `id` INT NOT NULL,
   `counter` INT NOT NULL,
   `payload` [ROW<name STRING, age INT, flag BOOLEAN>],
-  `topic` METADATA VIRTUAL,
+  `topic` STRING METADATA VIRTUAL,
   `ts` AS [orig_ts - INTERVAL '60' MINUTE],
-  `orig_ts` METADATA FROM 'timestamp',
+  `orig_ts` TIMESTAMP(3) METADATA FROM 'timestamp',
   `proctime` AS [PROCTIME()],
   WATERMARK FOR `ts` AS [ts - INTERVAL '5' SECOND],
   CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED
diff --git 
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java
 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java
index ef92f8b39d0..58510130796 100644
--- 
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java
+++ 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java
@@ -388,9 +388,9 @@ class SchemaResolutionTest {
                                 + "  `id` INT NOT NULL COMMENT 'people id',\n"
                                 + "  `counter` INT NOT NULL,\n"
                                 + "  `payload` [ROW<name STRING, age INT, flag 
BOOLEAN>],\n"
-                                + "  `topic` METADATA VIRTUAL COMMENT 'kafka 
topic',\n"
+                                + "  `topic` STRING METADATA VIRTUAL COMMENT 
'kafka topic',\n"
                                 + "  `ts` AS [orig_ts - INTERVAL '60' MINUTE] 
COMMENT 'rowtime',\n"
-                                + "  `orig_ts` METADATA FROM 'timestamp' 
COMMENT 'the ''origin'' timestamp',\n"
+                                + "  `orig_ts` TIMESTAMP(3) METADATA FROM 
'timestamp' COMMENT 'the ''origin'' timestamp',\n"
                                 + "  `proctime` AS [PROCTIME()],\n"
                                 + "  WATERMARK FOR `ts` AS [ts - INTERVAL '5' 
SECOND],\n"
                                 + "  CONSTRAINT `primary_constraint` PRIMARY 
KEY (`id`) NOT ENFORCED,\n"
diff --git 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java
 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java
index a6cda6b760a..654e919dfe5 100644
--- 
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java
+++ 
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java
@@ -948,8 +948,7 @@ public final class Schema {
         @Override
         public String toString() {
             final StringBuilder sb = new StringBuilder();
-            sb.append(super.toString());
-            sb.append(" METADATA");
+            sb.append(String.format("%s %s METADATA", super.toString(), 
dataType.toString()));
             if (metadataKey != null) {
                 sb.append(" FROM '");
                 sb.append(EncodingUtils.escapeSingleQuotes(metadataKey));
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
index b1db977e241..5a8e49305cc 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
@@ -952,7 +952,7 @@ class SqlMaterializedTableNodeToOperationConverterTest
                                 + "  `c1` AS [CURRENT_TIMESTAMP],\n"
                                 + "  `a` BIGINT NOT NULL,\n"
                                 + "  `b` STRING,\n"
-                                + "  `topic` METADATA VIRTUAL COMMENT 'kafka 
topic',\n"
+                                + "  `topic` STRING METADATA VIRTUAL COMMENT 
'kafka topic',\n"
                                 + "  `c` INT,\n"
                                 + "  `d` STRING,\n"
                                 + "  WATERMARK FOR `c1` AS [`c1` - INTERVAL 
'1' SECOND],\n"

Reply via email to