You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2022/02/09 13:41:27 UTC

[flink] branch release-1.13 updated: [FLINK-26039][table-runtime] Fix the incorrect value getter in map unnest table function

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

twalthr pushed a commit to branch release-1.13
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.13 by this push:
     new ace49aa  [FLINK-26039][table-runtime] Fix the incorrect value getter in map unnest table function
ace49aa is described below

commit ace49aa807f831e209dacf0129240137fff266f1
Author: fornaix <fo...@gmail.com>
AuthorDate: Wed Feb 9 15:54:49 2022 +0800

    [FLINK-26039][table-runtime] Fix the incorrect value getter in map unnest table function
    
    This closes #18679.
---
 .../planner/runtime/batch/sql/UnnestITCase.scala     | 20 +++++++++++++++++++-
 .../table/runtime/functions/SqlUnnestUtils.java      |  2 +-
 2 files changed, 20 insertions(+), 2 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala
index 3300e0a..9abd98c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.batch.sql
+package org.apache.flink.table.planner.runtime.batch.sql
 
 import org.apache.flink.api.java.typeutils.{ObjectArrayTypeInfo, RowTypeInfo}
 import org.apache.flink.table.api.Types
@@ -251,4 +251,22 @@ class UnnestITCase extends BatchTestBase {
     )
   }
 
+  @Test
+  def testUnnestMapWithDifferentKeyValueType(): Unit = {
+    val data = List(
+      row(1, Map("a" -> 10, "b" -> 11).asJava),
+      row(2, Map("c" -> 20, "d" -> 21).asJava)
+    )
+
+    registerCollection("T", data,
+      new RowTypeInfo(Types.INT, Types.MAP(Types.STRING, Types.INT)),
+      "a, b")
+
+    checkResult(
+      "SELECT a, k, v FROM T, UNNEST(T.b) as A(k, v)",
+      Seq(row(1, "a", 10), row(1, "b", 11), row(2, "c", 20),
+        row(2, "d", 21))
+    )
+  }
+
 }
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlUnnestUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlUnnestUtils.java
index b61801f..87b78b5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlUnnestUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlUnnestUtils.java
@@ -61,7 +61,7 @@ public final class SqlUnnestUtils {
                         mapType,
                         RowType.of(false, mapType.getKeyType(), mapType.getValueType()),
                         ArrayData.createElementGetter(mapType.getKeyType()),
-                        ArrayData.createElementGetter(mapType.getKeyType()));
+                        ArrayData.createElementGetter(mapType.getValueType()));
             default:
                 throw new UnsupportedOperationException("Unsupported type for UNNEST: " + t);
         }