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

dwysakowicz 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 f90740f  [FLINK-17420][table sql / api]Cannot alias Tuple and Row 
fields when converting DataStream to Table (#11951)11951)
f90740f is described below

commit f90740f951111c72b2b96d7b3adec3c940642e91
Author: Leonard Xu <[email protected]>
AuthorDate: Tue May 5 22:33:52 2020 +0800

    [FLINK-17420][table sql / api]Cannot alias Tuple and Row fields when 
converting DataStream to Table (#11951)11951)
---
 .../flink/table/typeutils/FieldInfoUtils.java      |   5 +
 .../flink/table/typeutils/FieldInfoUtilsTest.java  | 124 +++++++++++++++++++++
 2 files changed, 129 insertions(+)

diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java
index 3a3b0a0..85be133 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java
@@ -166,6 +166,11 @@ public class FieldInfoUtils {
                // This prevents confusing cases like ('f2, 'f0, 'myName) for a 
Tuple3 where fields are renamed
                // by position but the user might assume reordering instead of 
renaming.
                return Arrays.stream(fields).allMatch(f -> {
+                       if (f instanceof UnresolvedCallExpression &&
+                               ((UnresolvedCallExpression) 
f).getFunctionDefinition() == BuiltInFunctionDefinitions.AS &&
+                               f.getChildren().get(0) instanceof 
UnresolvedReferenceExpression) {
+                               return false;
+                       }
                        if (f instanceof UnresolvedReferenceExpression) {
                                return 
!inputNames.contains(((UnresolvedReferenceExpression) f).getName());
                        }
diff --git 
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/typeutils/FieldInfoUtilsTest.java
 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/typeutils/FieldInfoUtilsTest.java
new file mode 100644
index 0000000..ba22820
--- /dev/null
+++ 
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/typeutils/FieldInfoUtilsTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.typeutils.PojoField;
+import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.expressions.Expression;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.apache.flink.table.api.Expressions.$;
+
+/**
+ * Test suite for {@link FieldInfoUtils}.
+ */
+@RunWith(Enclosed.class)
+public class FieldInfoUtilsTest {
+
+       /**
+        * Test for ByNameMode.
+        */
+       @RunWith(Parameterized.class)
+       public static final class TestByNameMode {
+
+               @Parameterized.Parameters(name = "{0}")
+               public static Collection<TypeInformation> parameters() throws 
Exception {
+                       return Arrays.asList(
+                               new RowTypeInfo(
+                                       new TypeInformation[]{Types.INT, 
Types.LONG, Types.SQL_TIMESTAMP},
+                                       new String[]{"f0", "f1", "f2"}),
+                               new PojoTypeInfo(MyPojo.class, Arrays.asList(
+                                       new 
PojoField(MyPojo.class.getDeclaredField("f0"), Types.INT),
+                                       new 
PojoField(MyPojo.class.getDeclaredField("f1"), Types.LONG),
+                                       new 
PojoField(MyPojo.class.getDeclaredField("f2"), Types.SQL_TIMESTAMP))));
+               }
+
+               @Parameterized.Parameter
+               public TypeInformation typeInfo;
+
+               @Test
+               public void testByNameModeReorder() {
+                       FieldInfoUtils.TypeInfoSchema schema = 
FieldInfoUtils.getFieldsInfo(
+                               typeInfo,
+                               new Expression[]{$("f2"), $("f1"), $("f0")});
+
+                       Assert.assertArrayEquals(new String[]{"f2", "f1", 
"f0"}, schema.getFieldNames());
+               }
+
+               @Test
+               public void testByNameModeReorderAndRename() {
+                       FieldInfoUtils.TypeInfoSchema schema = 
FieldInfoUtils.getFieldsInfo(
+                               typeInfo,
+                               new Expression[]{$("f1").as("aa"), 
$("f0").as("bb"), $("f2").as("cc")});
+
+                       Assert.assertArrayEquals(new String[]{"aa", "bb", 
"cc"}, schema.getFieldNames());
+               }
+
+               /**
+                * Test Pojo class.
+                */
+               public static class MyPojo {
+                       public int f0;
+                       public long f1;
+                       public Timestamp f2;
+
+                       public MyPojo() {
+                       }
+               }
+       }
+
+       /**
+        * Test for ByPositionMode.
+        */
+       public static final class TestByPositionMode {
+               private static final RowTypeInfo rowTypeInfo = new RowTypeInfo(
+                       new TypeInformation[]{Types.INT, Types.LONG, 
Types.SQL_TIMESTAMP},
+                       new String[]{"f0", "f1", "f2"});
+
+               @Test
+               public void testByPositionMode() {
+                       FieldInfoUtils.TypeInfoSchema schema = 
FieldInfoUtils.getFieldsInfo(
+                               rowTypeInfo,
+                               new Expression[]{$("aa"), $("bb"), $("cc")});
+
+                       Assert.assertArrayEquals(new String[]{"aa", "bb", 
"cc"}, schema.getFieldNames());
+               }
+
+               @Test
+               public void testByPositionModeProcTime() {
+                       FieldInfoUtils.TypeInfoSchema schema = 
FieldInfoUtils.getFieldsInfo(
+                               rowTypeInfo,
+                               new Expression[]{$("aa"), $("bb"), $("cc"), 
$("cc").proctime().as("proctime")});
+
+                       Assert.assertArrayEquals(new String[]{"aa", "bb", "cc", 
"proctime"}, schema.getFieldNames());
+               }
+       }
+}

Reply via email to