From 76f475ed5469e83656cd75fd05780cfb9d05554b Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 29 Sep 2016 10:20:35 +0200 Subject: [PATCH 1/5] Merge FilterIT/SelectIT to CalcITCases --- .../{FilterITCase.java => CalcITCase.java} | 114 ++++++- .../api/java/batch/table/SelectITCase.java | 153 --------- .../{FilterITCase.scala => CalcITCase.scala} | 111 ++++++- .../api/scala/batch/sql/SelectITCase.scala | 148 --------- .../api/scala/batch/table/CalcITCase.scala | 297 +++++++++++++++++- .../api/scala/batch/table/FilterITCase.scala | 188 ----------- .../api/scala/batch/table/SelectITCase.scala | 190 ----------- .../{SelectITCase.scala => CalcITCase.scala} | 112 ++++++- .../api/scala/stream/table/FilterITCase.scala | 143 --------- 9 files changed, 628 insertions(+), 828 deletions(-) rename flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/{FilterITCase.java => CalcITCase.java} (57%) delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/{FilterITCase.scala => CalcITCase.scala} (57%) delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SelectITCase.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/{SelectITCase.scala => CalcITCase.scala} (62%) delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/FilterITCase.scala diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java similarity index 57% rename from flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java rename to flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java index 7a2bedf116362..3f74b7155c9fc 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java @@ -35,12 +35,122 @@ import java.util.List; @RunWith(Parameterized.class) -public class FilterITCase extends TableProgramsTestBase { +public class CalcITCase extends TableProgramsTestBase { - public FilterITCase(TestExecutionMode mode, TableConfigMode configMode){ + public CalcITCase(TestExecutionMode mode, TableConfigMode configMode){ super(mode, configMode); } + @Test + public void testSimpleSelectAllWithAs() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + Table in = tableEnv.fromDataSet(ds, "a,b,c"); + + Table result = in + .select("a, b, c"); + + DataSet resultSet = tableEnv.toDataSet(result, Row.class); + List results = resultSet.collect(); + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + + "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + + "20,6,Comment#14\n" + "21,6,Comment#15\n"; + compareResultAsText(results, expected); + } + + @Test + public void testSimpleSelectWithNaming() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + Table in = tableEnv.fromDataSet(ds); + + Table result = in + .select("f0 as a, f1 as b") + .select("a, b"); + + DataSet resultSet = tableEnv.toDataSet(result, Row.class); + List results = resultSet.collect(); + String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"; + compareResultAsText(results, expected); + } + + @Test + public void testSimpleSelectRenameAll() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + Table in = tableEnv.fromDataSet(ds); + + Table result = in + .select("f0 as a, f1 as b, f2 as c") + .select("a, b"); + + DataSet resultSet = tableEnv.toDataSet(result, Row.class); + List results = resultSet.collect(); + String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"; + compareResultAsText(results, expected); + } + + @Test(expected = ValidationException.class) + public void testSelectInvalidField() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + + tableEnv.fromDataSet(ds, "a, b, c") + // Must fail. Field foo does not exist + .select("a + 1, foo + 2"); + } + + @Test(expected = ValidationException.class) + public void testSelectAmbiguousFieldNames() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + + tableEnv.fromDataSet(ds, "a, b, c") + // Must fail. Field foo does not exist + .select("a + 1 as foo, b + 2 as foo"); + } + + @Test + public void testSelectStar() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> ds = CollectionDataSets.get3TupleDataSet(env); + Table in = tableEnv.fromDataSet(ds, "a,b,c"); + + Table result = in + .select("*"); + + DataSet resultSet = tableEnv.toDataSet(result, Row.class); + List results = resultSet.collect(); + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + + "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + + "20,6,Comment#14\n" + "21,6,Comment#15\n"; + compareResultAsText(results, expected); + } + @Test public void testAllRejectingFilter() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java deleted file mode 100644 index 581c8edf3b9de..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.api.java.batch.table; - -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.ValidationException; -import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -@RunWith(Parameterized.class) -public class SelectITCase extends TableProgramsTestBase { - - public SelectITCase(TestExecutionMode mode, TableConfigMode configMode) { - super(mode, configMode); - } - - @Test - public void testSimpleSelectAllWithAs() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - Table in = tableEnv.fromDataSet(ds, "a,b,c"); - - Table result = in - .select("a, b, c"); - - DataSet resultSet = tableEnv.toDataSet(result, Row.class); - List results = resultSet.collect(); - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + - "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + - "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + - "20,6,Comment#14\n" + "21,6,Comment#15\n"; - compareResultAsText(results, expected); - } - - @Test - public void testSimpleSelectWithNaming() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - Table in = tableEnv.fromDataSet(ds); - - Table result = in - .select("f0 as a, f1 as b") - .select("a, b"); - - DataSet resultSet = tableEnv.toDataSet(result, Row.class); - List results = resultSet.collect(); - String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + - "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + - "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"; - compareResultAsText(results, expected); - } - - @Test - public void testSimpleSelectRenameAll() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - Table in = tableEnv.fromDataSet(ds); - - Table result = in - .select("f0 as a, f1 as b, f2 as c") - .select("a, b"); - - DataSet resultSet = tableEnv.toDataSet(result, Row.class); - List results = resultSet.collect(); - String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + - "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + - "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"; - compareResultAsText(results, expected); - } - - @Test(expected = ValidationException.class) - public void testSelectInvalidField() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - tableEnv.fromDataSet(ds, "a, b, c") - // Must fail. Field foo does not exist - .select("a + 1, foo + 2"); - } - - @Test(expected = ValidationException.class) - public void testSelectAmbiguousFieldNames() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - - tableEnv.fromDataSet(ds, "a, b, c") - // Must fail. Field foo does not exist - .select("a + 1 as foo, b + 2 as foo"); - } - - @Test - public void testSelectStar() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - Table in = tableEnv.fromDataSet(ds, "a,b,c"); - - Table result = in - .select("*"); - - DataSet resultSet = tableEnv.toDataSet(result, Row.class); - List results = resultSet.collect(); - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + - "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + - "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + - "20,6,Comment#14\n" + "21,6,Comment#15\n"; - compareResultAsText(results, expected); - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/FilterITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala similarity index 57% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/FilterITCase.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala index cc4da3824abda..56045214a79f0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/FilterITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -33,11 +33,118 @@ import org.junit.runners.Parameterized import scala.collection.JavaConverters._ @RunWith(classOf[Parameterized]) -class FilterITCase( +class CalcITCase( mode: TestExecutionMode, configMode: TableConfigMode) extends TableProgramsTestBase(mode, configMode) { + @Test + def testSelectStarFromTable(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT * FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", ds) + + val result = tEnv.sql(sqlQuery) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSelectStarFromDataSet(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT * FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + + val result = tEnv.sql(sqlQuery) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSimpleSelectAll(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT a, b, c FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", ds) + + val result = tEnv.sql(sqlQuery) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSelectWithNaming(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT _1 as a, _2 as b FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) + tEnv.registerTable("MyTable", ds) + + val result = tEnv.sql(sqlQuery) + + val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" + + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test(expected = classOf[ValidationException]) + def testInvalidFields(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT a, foo FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", ds) + + tEnv.sql(sqlQuery) + } + @Test def testAllRejectingFilter(): Unit = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SelectITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SelectITCase.scala deleted file mode 100644 index 07b802d9945c3..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SelectITCase.scala +++ /dev/null @@ -1,148 +0,0 @@ -/* - * 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.api.scala.batch.sql - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{ValidationException, Row, TableEnvironment} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.TestBaseUtils -import org.junit._ -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class SelectITCase( - mode: TestExecutionMode, - configMode: TableConfigMode) - extends TableProgramsTestBase(mode, configMode) { - - @Test - def testSelectStarFromTable(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val sqlQuery = "SELECT * FROM MyTable" - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) - tEnv.registerTable("MyTable", ds) - - val result = tEnv.sql(sqlQuery) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - - val results = result.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSelectStarFromDataSet(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val sqlQuery = "SELECT * FROM MyTable" - - val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) - - val result = tEnv.sql(sqlQuery) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - - val results = result.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSimpleSelectAll(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val sqlQuery = "SELECT a, b, c FROM MyTable" - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) - tEnv.registerTable("MyTable", ds) - - val result = tEnv.sql(sqlQuery) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - - val results = result.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSelectWithNaming(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val sqlQuery = "SELECT _1 as a, _2 as b FROM MyTable" - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) - tEnv.registerTable("MyTable", ds) - - val result = tEnv.sql(sqlQuery) - - val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + - "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + - "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" - - val results = result.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test(expected = classOf[ValidationException]) - def testInvalidFields(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val sqlQuery = "SELECT a, foo FROM MyTable" - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) - tEnv.registerTable("MyTable", ds) - - tEnv.sql(sqlQuery) - } - -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala index d64e414049396..98c1cec641d7c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala @@ -23,9 +23,11 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.expressions.Literal +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils +import org.junit.Assert._ import org.junit._ import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -38,6 +40,299 @@ class CalcITCase( configMode: TableConfigMode) extends TableProgramsTestBase(mode, configMode) { + @Test + def testSimpleSelectAll(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).select('_1, '_2, '_3) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSimpleSelectAllWithAs(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('a, 'b, 'c) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSimpleSelectWithNaming(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) + .select('_1 as 'a, '_2 as 'b, '_1 as 'c) + .select('a, 'b) + + val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testSimpleSelectRenameAll(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) + .select('_1 as 'a, '_2 as 'b, '_3 as 'c) + .select('a, 'b) + + val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test(expected = classOf[ValidationException]) + def testSelectInvalidFieldFields(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + // must fail. Field 'foo does not exist + .select('a, 'foo) + } + + @Test(expected = classOf[ValidationException]) + def testSelectAmbiguousRenaming(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + // must fail. 'a and 'b are both renamed to 'foo + .select('a + 1 as 'foo, 'b + 2 as 'foo).toDataSet[Row].print() + } + + @Test(expected = classOf[ValidationException]) + def testSelectAmbiguousRenaming2(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + // must fail. 'a and 'b are both renamed to 'a + .select('a, 'b as 'a).toDataSet[Row].print() + } + + @Test + def testSelectStar(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testAliasStarException(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + try { + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, '*, 'b, 'c) + fail("ValidationException expected") + } catch { + case _: ValidationException => //ignore + } + + try { + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) + .select('_1 as '*, '_2 as 'b, '_1 as 'c) + fail("ValidationException expected") + } catch { + case _: ValidationException => //ignore + } + + try { + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('*, 'b, 'c) + fail("ValidationException expected") + } catch { + case _: ValidationException => //ignore + } + + try { + CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*, 'b) + fail("ValidationException expected") + } catch { + case _: ValidationException => //ignore + } + } + + @Test + def testAllRejectingFilter(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( Literal(false) ) + + val expected = "\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testAllPassingFilter(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( Literal(true) ) + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + + "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + + "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + + "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + + "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + + "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testFilterOnStringTupleField(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + val filterDs = ds.filter( 'c.like("%world%") ) + + val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testFilterOnIntegerTupleField(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( 'a % 2 === 0 ) + + val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + + "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" + + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," + + "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testNotEquals(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( 'a % 2 !== 0) + val expected = "1,1,Hi\n" + "3,2,Hello world\n" + + "5,3,I am fine.\n" + "7,4,Comment#1\n" + "9,4,Comment#3\n" + + "11,5,Comment#5\n" + "13,5,Comment#7\n" + "15,5,Comment#9\n" + + "17,6,Comment#11\n" + "19,6,Comment#13\n" + "21,6,Comment#15\n" + val results = filterDs.collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testDisjunctivePredicate(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( 'a < 2 || 'a > 20) + val expected = "1,1,Hi\n" + "21,6,Comment#15\n" + val results = filterDs.collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testConsecutiveFilters(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter('a % 2 !== 0).filter('b % 2 === 0) + val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" + + "9,4,Comment#3\n" + "17,6,Comment#11\n" + + "19,6,Comment#13\n" + "21,6,Comment#15\n" + val results = filterDs.collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testFilterBasicType(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.getStringDataSet(env) + + val filterDs = ds.toTable(tEnv, 'a).filter( 'a.like("H%") ) + + val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testFilterOnCustomType(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val filterDs = ds.toTable(tEnv, 'myInt as 'i, 'myLong as 'l, 'myString as 's) + .filter( 's.like("%a%") ) + + val expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test(expected = classOf[ValidationException]) + def testFilterInvalidFieldName(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + // must fail. Field 'foo does not exist + ds.filter( 'foo === 2 ) + } + @Test def testSimpleCalc(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala deleted file mode 100644 index ee0356f3a8957..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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.api.scala.batch.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.TestBaseUtils -import org.junit._ -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - - -@RunWith(classOf[Parameterized]) -class FilterITCase( - mode: TestExecutionMode, - configMode: TableConfigMode) - extends TableProgramsTestBase(mode, configMode) { - - @Test - def testAllRejectingFilter(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( Literal(false) ) - - val expected = "\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testAllPassingFilter(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( Literal(true) ) - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " + - "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," + - "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," + - "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," + - "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," + - "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testFilterOnStringTupleField(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val filterDs = ds.filter( 'c.like("%world%") ) - - val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testFilterOnIntegerTupleField(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( 'a % 2 === 0 ) - - val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + - "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" + - "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," + - "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testNotEquals(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( 'a % 2 !== 0) - val expected = "1,1,Hi\n" + "3,2,Hello world\n" + - "5,3,I am fine.\n" + "7,4,Comment#1\n" + "9,4,Comment#3\n" + - "11,5,Comment#5\n" + "13,5,Comment#7\n" + "15,5,Comment#9\n" + - "17,6,Comment#11\n" + "19,6,Comment#13\n" + "21,6,Comment#15\n" - val results = filterDs.collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testDisjunctivePredicate(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( 'a < 2 || 'a > 20) - val expected = "1,1,Hi\n" + "21,6,Comment#15\n" - val results = filterDs.collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testConsecutiveFilters(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter('a % 2 !== 0).filter('b % 2 === 0) - val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" + - "9,4,Comment#3\n" + "17,6,Comment#11\n" + - "19,6,Comment#13\n" + "21,6,Comment#15\n" - val results = filterDs.collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testFilterBasicType(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.getStringDataSet(env) - - val filterDs = ds.toTable(tEnv, 'a).filter( 'a.like("H%") ) - - val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testFilterOnCustomType(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.getCustomTypeDataSet(env) - val filterDs = ds.toTable(tEnv, 'myInt as 'i, 'myLong as 'l, 'myString as 's) - .filter( 's.like("%a%") ) - - val expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n" - val results = filterDs.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test(expected = classOf[ValidationException]) - def testFilterInvalidFieldName(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - - // must fail. Field 'foo does not exist - ds.filter( 'foo === 2 ) - } - -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala deleted file mode 100644 index 1143afdb97839..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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.api.scala.batch.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.TestBaseUtils -import org.junit.Assert._ -import org.junit._ -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class SelectITCase( - mode: TestExecutionMode, - configMode: TableConfigMode) - extends TableProgramsTestBase(mode, configMode) { - - @Test - def testSimpleSelectAll(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).select('_1, '_2, '_3) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSimpleSelectAllWithAs(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('a, 'b, 'c) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSimpleSelectWithNaming(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) - .select('_1 as 'a, '_2 as 'b, '_1 as 'c) - .select('a, 'b) - - val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + - "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + - "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testSimpleSelectRenameAll(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) - .select('_1 as 'a, '_2 as 'b, '_3 as 'c) - .select('a, 'b) - - val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" + - "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" + - "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test(expected = classOf[ValidationException]) - def testSelectInvalidFieldFields(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - // must fail. Field 'foo does not exist - .select('a, 'foo) - } - - @Test(expected = classOf[ValidationException]) - def testSelectAmbiguousRenaming(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - // must fail. 'a and 'b are both renamed to 'foo - .select('a + 1 as 'foo, 'b + 2 as 'foo).toDataSet[Row].print() - } - - @Test(expected = classOf[ValidationException]) - def testSelectAmbiguousRenaming2(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - // must fail. 'a and 'b are both renamed to 'a - .select('a, 'b as 'a).toDataSet[Row].print() - } - - @Test - def testSelectStar(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testAliasStarException(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - try { - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, '*, 'b, 'c) - fail("ValidationException expected") - } catch { - case _: ValidationException => //ignore - } - - try { - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) - .select('_1 as '*, '_2 as 'b, '_1 as 'c) - fail("ValidationException expected") - } catch { - case _: ValidationException => //ignore - } - - try { - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('*, 'b, 'c) - fail("ValidationException expected") - } catch { - case _: ValidationException => //ignore - } - - try { - CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*, 'b) - fail("ValidationException expected") - } catch { - case _: ValidationException => //ignore - } - } - -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala similarity index 62% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala index c6a2139d554d6..578ad30f2836f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala @@ -21,6 +21,7 @@ package org.apache.flink.api.scala.stream.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} import org.apache.flink.api.scala.table._ +import org.apache.flink.api.table.expressions.Literal import org.apache.flink.api.table.{Row, TableEnvironment, TableException} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase @@ -29,7 +30,7 @@ import org.junit.Test import scala.collection.mutable -class SelectITCase extends StreamingMultipleProgramsTestBase { +class CalcITCase extends StreamingMultipleProgramsTestBase { @Test def testSimpleSelectAll(): Unit = { @@ -172,4 +173,113 @@ class SelectITCase extends StreamingMultipleProgramsTestBase { val expected = mutable.MutableList("no") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + @Test + def testSimpleFilter(): Unit = { + /* + * Test simple filter + */ + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter('a === 3) + val results = filterDs.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList("3,2,Hello world") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + @Test + def testAllRejectingFilter(): Unit = { + /* + * Test all-rejecting filter + */ + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( Literal(false) ) + val results = filterDs.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + assertEquals(true, StreamITCase.testResults.isEmpty) + } + + @Test + def testAllPassingFilter(): Unit = { + /* + * Test all-passing filter + */ + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( Literal(true) ) + val results = filterDs.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "1,1,Hi", + "2,2,Hello", + "3,2,Hello world") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + @Test + def testFilterOnIntegerTupleField(): Unit = { + /* + * Test filter on Integer tuple field. + */ + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( 'a % 2 === 0 ) + val results = filterDs.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = mutable.MutableList( + "2,2,Hello", "4,3,Hello world, how are you?", + "6,3,Luke Skywalker", "8,4,Comment#2", "10,4,Comment#4", + "12,5,Comment#6", "14,5,Comment#8", "16,6,Comment#10", + "18,6,Comment#12", "20,6,Comment#14") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + @Test + def testNotEquals(): Unit = { + /* + * Test filter on Integer tuple field. + */ + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + + StreamITCase.testResults = mutable.MutableList() + val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) + + val filterDs = ds.filter( 'a % 2 !== 0) + val results = filterDs.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + val expected = mutable.MutableList( + "1,1,Hi", "3,2,Hello world", + "5,3,I am fine.", "7,4,Comment#1", "9,4,Comment#3", + "11,5,Comment#5", "13,5,Comment#7", "15,5,Comment#9", + "17,6,Comment#11", "19,6,Comment#13", "21,6,Comment#15") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/FilterITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/FilterITCase.scala deleted file mode 100644 index 45b9b04582f3b..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/FilterITCase.scala +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.api.scala.stream.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment} -import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment -import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase -import org.junit.Assert._ -import org.junit.Test - -import scala.collection.mutable - -class FilterITCase extends StreamingMultipleProgramsTestBase { - - @Test - def testSimpleFilter(): Unit = { - /* - * Test simple filter - */ - val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - StreamITCase.testResults = mutable.MutableList() - val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter('a === 3) - val results = filterDs.toDataStream[Row] - results.addSink(new StreamITCase.StringSink) - env.execute() - - val expected = mutable.MutableList("3,2,Hello world") - assertEquals(expected.sorted, StreamITCase.testResults.sorted) - } - - @Test - def testAllRejectingFilter(): Unit = { - /* - * Test all-rejecting filter - */ - val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - StreamITCase.testResults = mutable.MutableList() - val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( Literal(false) ) - val results = filterDs.toDataStream[Row] - results.addSink(new StreamITCase.StringSink) - env.execute() - - assertEquals(true, StreamITCase.testResults.isEmpty) - } - - @Test - def testAllPassingFilter(): Unit = { - /* - * Test all-passing filter - */ - val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - StreamITCase.testResults = mutable.MutableList() - val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( Literal(true) ) - val results = filterDs.toDataStream[Row] - results.addSink(new StreamITCase.StringSink) - env.execute() - - val expected = mutable.MutableList( - "1,1,Hi", - "2,2,Hello", - "3,2,Hello world") - assertEquals(expected.sorted, StreamITCase.testResults.sorted) - } - - @Test - def testFilterOnIntegerTupleField(): Unit = { - /* - * Test filter on Integer tuple field. - */ - val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - StreamITCase.testResults = mutable.MutableList() - val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( 'a % 2 === 0 ) - val results = filterDs.toDataStream[Row] - results.addSink(new StreamITCase.StringSink) - env.execute() - - val expected = mutable.MutableList( - "2,2,Hello", "4,3,Hello world, how are you?", - "6,3,Luke Skywalker", "8,4,Comment#2", "10,4,Comment#4", - "12,5,Comment#6", "14,5,Comment#8", "16,6,Comment#10", - "18,6,Comment#12", "20,6,Comment#14") - assertEquals(expected.sorted, StreamITCase.testResults.sorted) - } - - @Test - def testNotEquals(): Unit = { - /* - * Test filter on Integer tuple field. - */ - val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - StreamITCase.testResults = mutable.MutableList() - val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) - - val filterDs = ds.filter( 'a % 2 !== 0) - val results = filterDs.toDataStream[Row] - results.addSink(new StreamITCase.StringSink) - env.execute() - val expected = mutable.MutableList( - "1,1,Hi", "3,2,Hello world", - "5,3,I am fine.", "7,4,Comment#1", "9,4,Comment#3", - "11,5,Comment#5", "13,5,Comment#7", "15,5,Comment#9", - "17,6,Comment#11", "19,6,Comment#13", "21,6,Comment#15") - assertEquals(expected.sorted, StreamITCase.testResults.sorted) - } -} From e36bc8002d60f1cb742dd02949b2fbcac0052dbf Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 29 Sep 2016 11:04:34 +0200 Subject: [PATCH 2/5] Merge FromDataSet/ToTable to TableEnvironmentITCases --- .../java/batch/TableEnvironmentITCase.java | 474 ++++++++++++++++- .../java/batch/table/FromDataSetITCase.java | 499 ------------------ .../scala/batch/TableEnvironmentITCase.scala | 130 +++++ .../api/scala/batch/table/ToTableITCase.scala | 158 ------ .../batch/utils/TableProgramsTestBase.scala | 22 +- 5 files changed, 608 insertions(+), 675 deletions(-) delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java index 8fdb2da27823b..da66387db6816 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java @@ -18,20 +18,31 @@ package org.apache.flink.api.java.batch; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.*; +import org.apache.flink.api.table.Row; +import org.apache.flink.api.table.Table; +import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.TableException; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.List; - @RunWith(Parameterized.class) public class TableEnvironmentITCase extends TableProgramsTestBase { @@ -39,6 +50,15 @@ public TableEnvironmentITCase(TestExecutionMode mode, TableConfigMode configMode super(mode, configMode); } + @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}") + public static Collection parameters() { + return Arrays.asList(new Object[][] { + { TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() }, + { TestExecutionMode.COLLECTION, TableProgramsTestBase.NULL() }, + { TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT() } + }); + } + @Test public void testSimpleRegister() throws Exception { final String tableName = "MyTable"; @@ -145,4 +165,452 @@ public void testRegisterTableFromOtherEnv() throws Exception { // Must fail. Table is bound to different TableEnvironment. tableEnv2.registerTable("MyTable", t); } + + @Test + public void testAsFromTuple() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + Table table = tableEnv + .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") + .select("a, b, c"); + + DataSet ds = tableEnv.toDataSet(table, Row.class); + List results = ds.collect(); + String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + + "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + + "20,6,Comment#14\n" + "21,6,Comment#15\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromAndToTuple() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + Table table = tableEnv + .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") + .select("a, b, c"); + + TypeInformation ti = new TupleTypeInfo>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + DataSet ds = tableEnv.toDataSet(table, ti); + List results = ds.collect(); + String expected = "(1,1,Hi)\n" + "(2,2,Hello)\n" + "(3,2,Hello world)\n" + + "(4,3,Hello world, how are you?)\n" + "(5,3,I am fine.)\n" + "(6,3,Luke Skywalker)\n" + + "(7,4,Comment#1)\n" + "(8,4,Comment#2)\n" + "(9,4,Comment#3)\n" + "(10,4,Comment#4)\n" + + "(11,5,Comment#5)\n" + "(12,5,Comment#6)\n" + "(13,5,Comment#7)\n" + + "(14,5,Comment#8)\n" + "(15,5,Comment#9)\n" + "(16,6,Comment#10)\n" + + "(17,6,Comment#11)\n" + "(18,6,Comment#12)\n" + "(19,6,Comment#13)\n" + + "(20,6,Comment#14)\n" + "(21,6,Comment#15)\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromTupleToPojo() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List> data = new ArrayList<>(); + data.add(new Tuple4<>("Rofl", 1, 1.0, "Hi")); + data.add(new Tuple4<>("lol", 2, 1.0, "Hi")); + data.add(new Tuple4<>("Test me", 4, 3.33, "Hello world")); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), "a, b, c, d") + .select("a, b, c, d"); + + DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); + List results = ds.collect(); + String expected = "Rofl,1,1.0,Hi\n" + "lol,2,1.0,Hi\n" + "Test me,4,3.33,Hello world\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromPojo() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List data = new ArrayList<>(); + data.add(new SmallPojo("Peter", 28, 4000.00, "Sales")); + data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering")); + data.add(new SmallPojo("Lucy", 42, 6000.00, "HR")); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), + "department AS a, " + + "age AS b, " + + "salary AS c, " + + "name AS d") + .select("a, b, c, d"); + + DataSet ds = tableEnv.toDataSet(table, Row.class); + List results = ds.collect(); + String expected = + "Sales,28,4000.0,Peter\n" + + "Engineering,56,10000.0,Anna\n" + + "HR,42,6000.0,Lucy\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromPrivateFieldsPojo() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List data = new ArrayList<>(); + data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales")); + data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering")); + data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR")); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), + "department AS a, " + + "age AS b, " + + "salary AS c, " + + "name AS d") + .select("a, b, c, d"); + + DataSet ds = tableEnv.toDataSet(table, Row.class); + List results = ds.collect(); + String expected = + "Sales,28,4000.0,Peter\n" + + "Engineering,56,10000.0,Anna\n" + + "HR,42,6000.0,Lucy\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromAndToPojo() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List data = new ArrayList<>(); + data.add(new SmallPojo("Peter", 28, 4000.00, "Sales")); + data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering")); + data.add(new SmallPojo("Lucy", 42, 6000.00, "HR")); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), + "department AS a, " + + "age AS b, " + + "salary AS c, " + + "name AS d") + .select("a, b, c, d"); + + DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); + List results = ds.collect(); + String expected = + "Sales,28,4000.0,Peter\n" + + "Engineering,56,10000.0,Anna\n" + + "HR,42,6000.0,Lucy\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsFromAndToPrivateFieldPojo() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List data = new ArrayList<>(); + data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales")); + data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering")); + data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR")); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), + "department AS a, " + + "age AS b, " + + "salary AS c, " + + "name AS d") + .select("a, b, c, d"); + + DataSet ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class); + List results = ds.collect(); + String expected = + "Sales,28,4000.0,Peter\n" + + "Engineering,56,10000.0,Anna\n" + + "HR,42,6000.0,Lucy\n"; + compareResultAsText(results, expected); + } + + @Test + public void testAsWithPojoAndGenericTypes() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + List data = new ArrayList<>(); + data.add(new PojoWithGeneric("Peter", 28, new HashMap(), new ArrayList())); + HashMap hm1 = new HashMap<>(); + hm1.put("test1", "test1"); + data.add(new PojoWithGeneric("Anna", 56, hm1, new ArrayList())); + HashMap hm2 = new HashMap<>(); + hm2.put("abc", "cde"); + data.add(new PojoWithGeneric("Lucy", 42, hm2, new ArrayList())); + + Table table = tableEnv + .fromDataSet(env.fromCollection(data), + "name AS a, " + + "age AS b, " + + "generic AS c, " + + "generic2 AS d") + .select("a, b, c, c as c2, d") + .select("a, b, c, c === c2, d"); + + DataSet ds = tableEnv.toDataSet(table, Row.class); + List results = ds.collect(); + String expected = + "Peter,28,{},true,[]\n" + + "Anna,56,{test1=test1},true,[]\n" + + "Lucy,42,{abc=cde},true,[]\n"; + compareResultAsText(results, expected); + } + + @Test(expected = TableException.class) + public void testAsWithToFewFields() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail. Not enough field names specified. + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b"); + } + + @Test(expected = TableException.class) + public void testAsWithToManyFields() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail. Too many field names specified. + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d"); + } + + @Test(expected = TableException.class) + public void testAsWithAmbiguousFields() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail. Specified field names are not unique. + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b"); + } + + @Test(expected = TableException.class) + public void testAsWithNonFieldReference1() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail. as() does only allow field name expressions + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c"); + } + + @Test(expected = TableException.class) + public void testAsWithNonFieldReference2() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail. as() does only allow field name expressions + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a as foo, b, c"); + } + + @Test(expected = TableException.class) + public void testNonStaticClassInput() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail since class is not static + tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), "name"); + } + + @Test(expected = TableException.class) + public void testNonStaticClassOutput() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + // Must fail since class is not static + Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), "number"); + tableEnv.toDataSet(t, MyNonStatic.class); + } + + // -------------------------------------------------------------------------------------------- + + public class MyNonStatic { + public int number; + } + + @SuppressWarnings("unused") + public static class SmallPojo { + + public SmallPojo() { } + + public SmallPojo(String name, int age, double salary, String department) { + this.name = name; + this.age = age; + this.salary = salary; + this.department = department; + } + + public String name; + public int age; + public double salary; + public String department; + } + + @SuppressWarnings("unused") + public static class PojoWithGeneric { + public String name; + public int age; + public HashMap generic; + public ArrayList generic2; + + public PojoWithGeneric() { + // default constructor + } + + public PojoWithGeneric(String name, int age, HashMap generic, + ArrayList generic2) { + this.name = name; + this.age = age; + this.generic = generic; + this.generic2 = generic2; + } + + @Override + public String toString() { + return name + "," + age + "," + generic + "," + generic2; + } + } + + @SuppressWarnings("unused") + public static class PrivateSmallPojo { + + public PrivateSmallPojo() { } + + public PrivateSmallPojo(String name, int age, double salary, String department) { + this.name = name; + this.age = age; + this.salary = salary; + this.department = department; + } + + private String name; + private int age; + private double salary; + private String department; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + + public double getSalary() { + return salary; + } + + public void setSalary(double salary) { + this.salary = salary; + } + + public String getDepartment() { + return department; + } + + public void setDepartment(String department) { + this.department = department; + } + } + + @SuppressWarnings("unused") + public static class SmallPojo2 { + + public SmallPojo2() { } + + public SmallPojo2(String a, int b, double c, String d) { + this.a = a; + this.b = b; + this.c = c; + this.d = d; + } + + public String a; + public int b; + public double c; + public String d; + + @Override + public String toString() { + return a + "," + b + "," + c + "," + d; + } + } + + @SuppressWarnings("unused") + public static class PrivateSmallPojo2 { + + public PrivateSmallPojo2() { } + + public PrivateSmallPojo2(String a, int b, double c, String d) { + this.a = a; + this.b = b; + this.c = c; + this.d = d; + } + + private String a; + private int b; + private double c; + private String d; + + public String getA() { + return a; + } + + public void setA(String a) { + this.a = a; + } + + public int getB() { + return b; + } + + public void setB(int b) { + this.b = b; + } + + public double getC() { + return c; + } + + public void setC(double c) { + this.c = c; + } + + public String getD() { + return d; + } + + public void setD(String d) { + this.d = d; + } + + @Override + public String toString() { + return a + "," + b + "," + c + "," + d; + } + } } diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java deleted file mode 100644 index e6b92262fa3f5..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java +++ /dev/null @@ -1,499 +0,0 @@ -/* - * 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.api.java.batch.table; - -import java.util.HashMap; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.TableException; -import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.ArrayList; -import java.util.List; - -@RunWith(Parameterized.class) -public class FromDataSetITCase extends TableProgramsTestBase { - - public FromDataSetITCase(TestExecutionMode mode, TableConfigMode configMode){ - super(mode, configMode); - } - - @Test - public void testAsFromTuple() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - Table table = tableEnv - .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") - .select("a, b, c"); - - DataSet ds = tableEnv.toDataSet(table, Row.class); - List results = ds.collect(); - String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + - "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + - "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" + - "20,6,Comment#14\n" + "21,6,Comment#15\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromAndToTuple() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - Table table = tableEnv - .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") - .select("a, b, c"); - - TypeInformation ti = new TupleTypeInfo>( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO); - - DataSet ds = tableEnv.toDataSet(table, ti); - List results = ds.collect(); - String expected = "(1,1,Hi)\n" + "(2,2,Hello)\n" + "(3,2,Hello world)\n" + - "(4,3,Hello world, how are you?)\n" + "(5,3,I am fine.)\n" + "(6,3,Luke Skywalker)\n" + - "(7,4,Comment#1)\n" + "(8,4,Comment#2)\n" + "(9,4,Comment#3)\n" + "(10,4,Comment#4)\n" + - "(11,5,Comment#5)\n" + "(12,5,Comment#6)\n" + "(13,5,Comment#7)\n" + - "(14,5,Comment#8)\n" + "(15,5,Comment#9)\n" + "(16,6,Comment#10)\n" + - "(17,6,Comment#11)\n" + "(18,6,Comment#12)\n" + "(19,6,Comment#13)\n" + - "(20,6,Comment#14)\n" + "(21,6,Comment#15)\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromTupleToPojo() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List> data = new ArrayList<>(); - data.add(new Tuple4<>("Rofl", 1, 1.0, "Hi")); - data.add(new Tuple4<>("lol", 2, 1.0, "Hi")); - data.add(new Tuple4<>("Test me", 4, 3.33, "Hello world")); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), "a, b, c, d") - .select("a, b, c, d"); - - DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); - List results = ds.collect(); - String expected = "Rofl,1,1.0,Hi\n" + "lol,2,1.0,Hi\n" + "Test me,4,3.33,Hello world\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromPojo() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List data = new ArrayList<>(); - data.add(new SmallPojo("Peter", 28, 4000.00, "Sales")); - data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering")); - data.add(new SmallPojo("Lucy", 42, 6000.00, "HR")); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") - .select("a, b, c, d"); - - DataSet ds = tableEnv.toDataSet(table, Row.class); - List results = ds.collect(); - String expected = - "Sales,28,4000.0,Peter\n" + - "Engineering,56,10000.0,Anna\n" + - "HR,42,6000.0,Lucy\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromPrivateFieldsPojo() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List data = new ArrayList<>(); - data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales")); - data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering")); - data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR")); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") - .select("a, b, c, d"); - - DataSet ds = tableEnv.toDataSet(table, Row.class); - List results = ds.collect(); - String expected = - "Sales,28,4000.0,Peter\n" + - "Engineering,56,10000.0,Anna\n" + - "HR,42,6000.0,Lucy\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromAndToPojo() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List data = new ArrayList<>(); - data.add(new SmallPojo("Peter", 28, 4000.00, "Sales")); - data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering")); - data.add(new SmallPojo("Lucy", 42, 6000.00, "HR")); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") - .select("a, b, c, d"); - - DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); - List results = ds.collect(); - String expected = - "Sales,28,4000.0,Peter\n" + - "Engineering,56,10000.0,Anna\n" + - "HR,42,6000.0,Lucy\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsFromAndToPrivateFieldPojo() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List data = new ArrayList<>(); - data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales")); - data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering")); - data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR")); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") - .select("a, b, c, d"); - - DataSet ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class); - List results = ds.collect(); - String expected = - "Sales,28,4000.0,Peter\n" + - "Engineering,56,10000.0,Anna\n" + - "HR,42,6000.0,Lucy\n"; - compareResultAsText(results, expected); - } - - @Test - public void testAsWithPojoAndGenericTypes() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - List data = new ArrayList<>(); - data.add(new PojoWithGeneric("Peter", 28, new HashMap(), new ArrayList())); - HashMap hm1 = new HashMap<>(); - hm1.put("test1", "test1"); - data.add(new PojoWithGeneric("Anna", 56, hm1, new ArrayList())); - HashMap hm2 = new HashMap<>(); - hm2.put("abc", "cde"); - data.add(new PojoWithGeneric("Lucy", 42, hm2, new ArrayList())); - - Table table = tableEnv - .fromDataSet(env.fromCollection(data), - "name AS a, " + - "age AS b, " + - "generic AS c, " + - "generic2 AS d") - .select("a, b, c, c as c2, d") - .select("a, b, c, c === c2, d"); - - DataSet ds = tableEnv.toDataSet(table, Row.class); - List results = ds.collect(); - String expected = - "Peter,28,{},true,[]\n" + - "Anna,56,{test1=test1},true,[]\n" + - "Lucy,42,{abc=cde},true,[]\n"; - compareResultAsText(results, expected); - } - - @Test(expected = TableException.class) - public void testAsWithToFewFields() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail. Not enough field names specified. - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b"); - } - - @Test(expected = TableException.class) - public void testAsWithToManyFields() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail. Too many field names specified. - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d"); - } - - @Test(expected = TableException.class) - public void testAsWithAmbiguousFields() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail. Specified field names are not unique. - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b"); - } - - @Test(expected = TableException.class) - public void testAsWithNonFieldReference1() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail. as() does only allow field name expressions - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c"); - } - - @Test(expected = TableException.class) - public void testAsWithNonFieldReference2() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail. as() does only allow field name expressions - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a as foo, b, c"); - } - - @Test(expected = TableException.class) - public void testNonStaticClassInput() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail since class is not static - tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), "name"); - } - - @Test(expected = TableException.class) - public void testNonStaticClassOutput() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); - - // Must fail since class is not static - Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), "number"); - tableEnv.toDataSet(t, MyNonStatic.class); - } - - // -------------------------------------------------------------------------------------------- - - public class MyNonStatic { - public int number; - } - - @SuppressWarnings("unused") - public static class SmallPojo { - - public SmallPojo() { } - - public SmallPojo(String name, int age, double salary, String department) { - this.name = name; - this.age = age; - this.salary = salary; - this.department = department; - } - - public String name; - public int age; - public double salary; - public String department; - } - - @SuppressWarnings("unused") - public static class PojoWithGeneric { - public String name; - public int age; - public HashMap generic; - public ArrayList generic2; - - public PojoWithGeneric() { - // default constructor - } - - public PojoWithGeneric(String name, int age, HashMap generic, - ArrayList generic2) { - this.name = name; - this.age = age; - this.generic = generic; - this.generic2 = generic2; - } - - @Override - public String toString() { - return name + "," + age + "," + generic + "," + generic2; - } - } - - @SuppressWarnings("unused") - public static class PrivateSmallPojo { - - public PrivateSmallPojo() { } - - public PrivateSmallPojo(String name, int age, double salary, String department) { - this.name = name; - this.age = age; - this.salary = salary; - this.department = department; - } - - private String name; - private int age; - private double salary; - private String department; - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - - public double getSalary() { - return salary; - } - - public void setSalary(double salary) { - this.salary = salary; - } - - public String getDepartment() { - return department; - } - - public void setDepartment(String department) { - this.department = department; - } - } - - @SuppressWarnings("unused") - public static class SmallPojo2 { - - public SmallPojo2() { } - - public SmallPojo2(String a, int b, double c, String d) { - this.a = a; - this.b = b; - this.c = c; - this.d = d; - } - - public String a; - public int b; - public double c; - public String d; - - @Override - public String toString() { - return a + "," + b + "," + c + "," + d; - } - } - - @SuppressWarnings("unused") - public static class PrivateSmallPojo2 { - - public PrivateSmallPojo2() { } - - public PrivateSmallPojo2(String a, int b, double c, String d) { - this.a = a; - this.b = b; - this.c = c; - this.d = d; - } - - private String a; - private int b; - private double c; - private String d; - - public String getA() { - return a; - } - - public void setA(String a) { - this.a = a; - } - - public int getB() { - return b; - } - - public void setB(int b) { - this.b = b; - } - - public double getC() { - return c; - } - - public void setC(double c) { - this.c = c; - } - - public String getD() { - return d; - } - - public void setD(String d) { - this.d = d; - } - - @Override - public String toString() { - return a + "," + b + "," + c + "," + d; - } - } - -} - diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala index c33e1ef925a42..36b2969603bbe 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala @@ -18,6 +18,8 @@ package org.apache.flink.api.scala.batch +import java.util + import org.apache.flink.api.scala._ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode @@ -140,4 +142,132 @@ class TableEnvironmentITCase( // Must fail. Table is bound to different TableEnvironment. tEnv2.registerTable("MyTable", t1) } + + @Test + def testToTable(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env) + .toTable(tEnv, 'a, 'b, 'c) + .select('a, 'b, 'c) + + val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testToTableFromCaseClass(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val data = List( + SomeCaseClass("Peter", 28, 4000.00, "Sales"), + SomeCaseClass("Anna", 56, 10000.00, "Engineering"), + SomeCaseClass("Lucy", 42, 6000.00, "HR")) + + val t = env.fromCollection(data) + .toTable(tEnv, 'a, 'b, 'c, 'd) + .select('a, 'b, 'c, 'd) + + val expected: String = + "Peter,28,4000.0,Sales\n" + + "Anna,56,10000.0,Engineering\n" + + "Lucy,42,6000.0,HR\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testToTableFromAndToCaseClass(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val data = List( + SomeCaseClass("Peter", 28, 4000.00, "Sales"), + SomeCaseClass("Anna", 56, 10000.00, "Engineering"), + SomeCaseClass("Lucy", 42, 6000.00, "HR")) + + val t = env.fromCollection(data) + .toTable(tEnv, 'a, 'b, 'c, 'd) + .select('a, 'b, 'c, 'd) + + val expected: String = + "SomeCaseClass(Peter,28,4000.0,Sales)\n" + + "SomeCaseClass(Anna,56,10000.0,Engineering)\n" + + "SomeCaseClass(Lucy,42,6000.0,HR)\n" + val results = t.toDataSet[SomeCaseClass].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test(expected = classOf[TableException]) + def testToTableWithToFewFields(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env) + // Must fail. Number of fields does not match. + .toTable(tEnv, 'a, 'b) + } + + @Test(expected = classOf[TableException]) + def testToTableWithToManyFields(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env) + // Must fail. Number of fields does not match. + .toTable(tEnv, 'a, 'b, 'c, 'd) + } + + @Test(expected = classOf[TableException]) + def testToTableWithAmbiguousFields(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + CollectionDataSets.get3TupleDataSet(env) + // Must fail. Field names not unique. + .toTable(tEnv, 'a, 'b, 'b) + } + + @Test(expected = classOf[TableException]) + def testToTableWithNonFieldReference1(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + // Must fail. as() can only have field references + CollectionDataSets.get3TupleDataSet(env) + .toTable(tEnv, 'a + 1, 'b, 'c) + } + + @Test(expected = classOf[TableException]) + def testToTableWithNonFieldReference2(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + // Must fail. as() can only have field references + CollectionDataSets.get3TupleDataSet(env) + .toTable(tEnv, 'a as 'foo, 'b, 'c) + } +} + +object TableEnvironmentITCase { + + @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}") + def parameters(): util.Collection[Array[java.lang.Object]] = { + Seq[Array[AnyRef]]( + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT), + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT), + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT)).asJava + } +} + +case class SomeCaseClass(name: String, age: Int, salary: Double, department: String) { + def this() { this("", 0, 0.0, "") } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala deleted file mode 100644 index 84bdbb0e9ec36..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.api.scala.batch.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.TestBaseUtils -import org.junit._ -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class ToTableITCase( - mode: TestExecutionMode, - configMode: TableConfigMode) - extends TableProgramsTestBase(mode, configMode) { - - @Test - def testToTable(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val t = CollectionDataSets.get3TupleDataSet(env) - .toTable(tEnv, 'a, 'b, 'c) - .select('a, 'b, 'c) - - val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + - "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + - "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + - "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + - "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + - "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testToTableFromCaseClass(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val data = List( - SomeCaseClass("Peter", 28, 4000.00, "Sales"), - SomeCaseClass("Anna", 56, 10000.00, "Engineering"), - SomeCaseClass("Lucy", 42, 6000.00, "HR")) - - val t = env.fromCollection(data) - .toTable(tEnv, 'a, 'b, 'c, 'd) - .select('a, 'b, 'c, 'd) - - val expected: String = - "Peter,28,4000.0,Sales\n" + - "Anna,56,10000.0,Engineering\n" + - "Lucy,42,6000.0,HR\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testToTableFromAndToCaseClass(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - val data = List( - SomeCaseClass("Peter", 28, 4000.00, "Sales"), - SomeCaseClass("Anna", 56, 10000.00, "Engineering"), - SomeCaseClass("Lucy", 42, 6000.00, "HR")) - - val t = env.fromCollection(data) - .toTable(tEnv, 'a, 'b, 'c, 'd) - .select('a, 'b, 'c, 'd) - - val expected: String = - "SomeCaseClass(Peter,28,4000.0,Sales)\n" + - "SomeCaseClass(Anna,56,10000.0,Engineering)\n" + - "SomeCaseClass(Lucy,42,6000.0,HR)\n" - val results = t.toDataSet[SomeCaseClass].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test(expected = classOf[TableException]) - def testToTableWithToFewFields(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env) - // Must fail. Number of fields does not match. - .toTable(tEnv, 'a, 'b) - } - - @Test(expected = classOf[TableException]) - def testToTableWithToManyFields(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env) - // Must fail. Number of fields does not match. - .toTable(tEnv, 'a, 'b, 'c, 'd) - } - - @Test(expected = classOf[TableException]) - def testToTableWithAmbiguousFields(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - CollectionDataSets.get3TupleDataSet(env) - // Must fail. Field names not unique. - .toTable(tEnv, 'a, 'b, 'b) - } - - @Test(expected = classOf[TableException]) - def testToTableWithNonFieldReference1(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - // Must fail. as() can only have field references - CollectionDataSets.get3TupleDataSet(env) - .toTable(tEnv, 'a + 1, 'b, 'c) - } - - @Test(expected = classOf[TableException]) - def testToTableWithNonFieldReference2(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - // Must fail. as() can only have field references - CollectionDataSets.get3TupleDataSet(env) - .toTable(tEnv, 'a as 'foo, 'b, 'c) - } - -} - -case class SomeCaseClass(name: String, age: Int, salary: Double, department: String) { - def this() { this("", 0, 0.0, "") } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala index 772850d3cf29f..becfade8dc8b4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala @@ -20,8 +20,7 @@ package org.apache.flink.api.scala.batch.utils import java.util -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode._ +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.{EFFICIENT, NULL, TableConfigMode} import org.apache.flink.api.table.TableConfig import org.apache.flink.test.util.MultipleProgramsTestBase import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode @@ -48,21 +47,14 @@ class TableProgramsTestBase( } object TableProgramsTestBase { - sealed trait TableConfigMode { def nullCheck: Boolean; def efficientTypes: Boolean } - object TableConfigMode { - case object DEFAULT extends TableConfigMode { - val nullCheck = false; val efficientTypes = false - } - case object NULL extends TableConfigMode { - val nullCheck = true; val efficientTypes = false - } - case object EFFICIENT extends TableConfigMode { - val nullCheck = false; val efficientTypes = true - } - } + case class TableConfigMode(nullCheck: Boolean, efficientTypes: Boolean) + + val DEFAULT = TableConfigMode(nullCheck = false, efficientTypes = false) + val NULL = TableConfigMode(nullCheck = true, efficientTypes = false) + val EFFICIENT = TableConfigMode(nullCheck = false, efficientTypes = true) @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}") def parameters(): util.Collection[Array[java.lang.Object]] = { - Seq[Array[AnyRef]](Array(TestExecutionMode.COLLECTION, TableConfigMode.DEFAULT)) + Seq[Array[AnyRef]](Array(TestExecutionMode.COLLECTION, DEFAULT)) } } From 4810450322c9b56b1e59ff5469a5cfdc91abae9a Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 29 Sep 2016 11:22:16 +0200 Subject: [PATCH 3/5] Merge aggregating ITCases --- .../java/batch/table/AggregationsITCase.java | 194 ++++++++++++++- .../api/java/batch/table/DistinctITCase.java | 76 ------ .../table/GroupedAggregationsITCase.java | 124 ---------- .../java/batch/table/PojoGroupingITCase.java | 90 ------- .../batch/table/AggregationsITCase.scala | 222 +++++++++++++++++- .../scala/batch/table/DistinctITCase.scala | 62 ----- .../table/GroupedAggregationsITCase.scala | 200 ---------------- 7 files changed, 392 insertions(+), 576 deletions(-) delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/DistinctITCase.java delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/PojoGroupingITCase.java delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/DistinctITCase.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java index 6bcac560c43ef..2b42b95554ab3 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java @@ -17,8 +17,15 @@ */ package org.apache.flink.api.java.batch.table; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.java.ExecutionEnvironment; @@ -38,16 +45,16 @@ import java.util.List; @RunWith(Parameterized.class) -public class AggregationsITCase extends MultipleProgramsTestBase { +public class AggregationsITCase extends TableProgramsTestBase { - public AggregationsITCase(TestExecutionMode mode){ - super(mode); + public AggregationsITCase(TestExecutionMode mode, TableConfigMode configMode){ + super(mode, configMode); } @Test public void testAggregationTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env)); @@ -62,7 +69,7 @@ public void testAggregationTypes() throws Exception { @Test(expected = ValidationException.class) public void testAggregationOnNonExistingField() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env)); @@ -79,7 +86,7 @@ public void testAggregationOnNonExistingField() throws Exception { @Test public void testWorkingAggregationDataTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSource> input = env.fromElements( @@ -100,7 +107,7 @@ public void testWorkingAggregationDataTypes() throws Exception { @Test public void testAggregationWithArithmetic() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSource> input = env.fromElements( @@ -122,7 +129,7 @@ public void testAggregationWithArithmetic() throws Exception { @Test public void testAggregationWithTwoCount() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSource> input = env.fromElements( @@ -144,7 +151,7 @@ public void testAggregationWithTwoCount() throws Exception { @Test(expected = ValidationException.class) public void testNonWorkingDataTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSource> input = env.fromElements(new Tuple2<>(1f, "Hello")); @@ -164,7 +171,7 @@ public void testNonWorkingDataTypes() throws Exception { @Test(expected = ValidationException.class) public void testNoNestedAggregation() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSource> input = env.fromElements(new Tuple2<>(1f, "Hello")); @@ -181,10 +188,90 @@ public void testNoNestedAggregation() throws Exception { compareResultAsText(results, expected); } + @Test(expected = ValidationException.class) + public void testGroupingOnNonExistentField() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + + tableEnv + .fromDataSet(input, "a, b, c") + // must fail. Field foo is not in input + .groupBy("foo") + .select("a.avg"); + } + + @Test(expected = ValidationException.class) + public void testGroupingInvalidSelection() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + + tableEnv + .fromDataSet(input, "a, b, c") + .groupBy("a, b") + // must fail. Field c is not a grouping key or aggregation + .select("c"); + } + + @Test + public void testGroupedAggregate() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + Table table = tableEnv.fromDataSet(input, "a, b, c"); + + Table result = table + .groupBy("b").select("b, a.sum"); + + DataSet ds = tableEnv.toDataSet(result, Row.class); + List results = ds.collect(); + String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; + compareResultAsText(results, expected); + } + + @Test + public void testGroupingKeyForwardIfNotUsed() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + Table table = tableEnv.fromDataSet(input, "a, b, c"); + + Table result = table + .groupBy("b").select("a.sum"); + + DataSet ds = tableEnv.toDataSet(result, Row.class); + List results = ds.collect(); + String expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n"; + compareResultAsText(results, expected); + } + + @Test + public void testGroupNoAggregation() throws Exception { + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + Table table = tableEnv.fromDataSet(input, "a, b, c"); + + Table result = table + .groupBy("b").select("a.sum as d, b").groupBy("b, d").select("b"); + + DataSet ds = tableEnv.toDataSet(result, Row.class); + String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; + List results = ds.collect(); + compareResultAsText(results, expected); + } + @Test public void testPojoAggregation() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); DataSet input = env.fromElements( new WC("Hello", 1), new WC("Ciao", 1), @@ -208,5 +295,90 @@ public String map(WC value) throws Exception { String expected = "Hello\n" + "Hola"; compareResultAsText(result, expected); } + + @Test + public void testPojoGrouping() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet> data = env.fromElements( + new Tuple3<>("A", 23.0, "Z"), + new Tuple3<>("A", 24.0, "Y"), + new Tuple3<>("B", 1.0, "Z")); + + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + Table table = tableEnv + .fromDataSet(data, "groupMe, value, name") + .select("groupMe, value, name") + .where("groupMe != 'B'"); + + DataSet myPojos = tableEnv.toDataSet(table, MyPojo.class); + + DataSet result = myPojos.groupBy("groupMe") + .sortGroup("value", Order.DESCENDING) + .first(1); + + List resultList = result.collect(); + compareResultAsText(resultList, "A,24.0,Y"); + } + + @Test + public void testDistinct() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get3TupleDataSet(env); + + Table table = tableEnv.fromDataSet(input, "a, b, c"); + + Table distinct = table.select("b").distinct(); + + DataSet ds = tableEnv.toDataSet(distinct, Row.class); + List results = ds.collect(); + String expected = "1\n" + "2\n" + "3\n"+ "4\n"+ "5\n"+ "6\n"; + compareResultAsText(results, expected); + } + + @Test + public void testDistinctAfterAggregate() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); + + DataSet> input = CollectionDataSets.get5TupleDataSet(env); + + Table table = tableEnv.fromDataSet(input, "a, b, c, d, e"); + + Table distinct = table.groupBy("a, e").select("e").distinct(); + + DataSet ds = tableEnv.toDataSet(distinct, Row.class); + List results = ds.collect(); + String expected = "1\n" + "2\n" + "3\n"; + compareResultAsText(results, expected); + } + + // -------------------------------------------------------------------------------------------- + + public static class MyPojo implements Serializable { + private static final long serialVersionUID = 8741918940120107213L; + + public String groupMe; + public double value; + public String name; + + public MyPojo() { + // for serialization + } + + public MyPojo(String groupMe, double value, String name) { + this.groupMe = groupMe; + this.value = value; + this.name = name; + } + + @Override + public String toString() { + return groupMe + "," + value + "," + name; + } + } } diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/DistinctITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/DistinctITCase.java deleted file mode 100644 index 7f10433bd97af..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/DistinctITCase.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.api.java.batch.table; - -import java.util.List; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class DistinctITCase extends MultipleProgramsTestBase { - - public DistinctITCase(TestExecutionMode mode){ - super(mode); - } - - @Test - public void testDistinct() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - - Table table = tableEnv.fromDataSet(input, "a, b, c"); - - Table distinct = table.select("b").distinct(); - - DataSet ds = tableEnv.toDataSet(distinct, Row.class); - List results = ds.collect(); - String expected = "1\n" + "2\n" + "3\n"+ "4\n"+ "5\n"+ "6\n"; - compareResultAsText(results, expected); - } - - @Test - public void testDistinctAfterAggregate() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get5TupleDataSet(env); - - Table table = tableEnv.fromDataSet(input, "a, b, c, d, e"); - - Table distinct = table.groupBy("a, e").select("e").distinct(); - - DataSet ds = tableEnv.toDataSet(distinct, Row.class); - List results = ds.collect(); - String expected = "1\n" + "2\n" + "3\n"; - compareResultAsText(results, expected); - } -} diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java deleted file mode 100644 index 19060404e3510..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.api.java.batch.table; - -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.ValidationException; -import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -@RunWith(Parameterized.class) -public class GroupedAggregationsITCase extends MultipleProgramsTestBase { - - public GroupedAggregationsITCase(TestExecutionMode mode){ - super(mode); - } - - @Test(expected = ValidationException.class) - public void testGroupingOnNonExistentField() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - - tableEnv - .fromDataSet(input, "a, b, c") - // must fail. Field foo is not in input - .groupBy("foo") - .select("a.avg"); - } - - @Test(expected = ValidationException.class) - public void testGroupingInvalidSelection() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - - tableEnv - .fromDataSet(input, "a, b, c") - .groupBy("a, b") - // must fail. Field c is not a grouping key or aggregation - .select("c"); - } - - @Test - public void testGroupedAggregate() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - Table table = tableEnv.fromDataSet(input, "a, b, c"); - - Table result = table - .groupBy("b").select("b, a.sum"); - - DataSet ds = tableEnv.toDataSet(result, Row.class); - List results = ds.collect(); - String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"; - compareResultAsText(results, expected); - } - - @Test - public void testGroupingKeyForwardIfNotUsed() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - Table table = tableEnv.fromDataSet(input, "a, b, c"); - - Table result = table - .groupBy("b").select("a.sum"); - - DataSet ds = tableEnv.toDataSet(result, Row.class); - List results = ds.collect(); - String expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n"; - compareResultAsText(results, expected); - } - - @Test - public void testGroupNoAggregation() throws Exception { - - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> input = CollectionDataSets.get3TupleDataSet(env); - Table table = tableEnv.fromDataSet(input, "a, b, c"); - - Table result = table - .groupBy("b").select("a.sum as d, b").groupBy("b, d").select("b"); - - DataSet ds = tableEnv.toDataSet(result, Row.class); - String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"; - List results = ds.collect(); - compareResultAsText(results, expected); - } -} - diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/PojoGroupingITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/PojoGroupingITCase.java deleted file mode 100644 index ba564bf5c8144..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/PojoGroupingITCase.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.api.java.batch.table; - -import java.io.Serializable; -import java.util.List; -import org.apache.flink.api.common.operators.Order; -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class PojoGroupingITCase extends MultipleProgramsTestBase { - - public PojoGroupingITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testPojoGrouping() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - DataSet> data = env.fromElements( - new Tuple3<>("A", 23.0, "Z"), - new Tuple3<>("A", 24.0, "Y"), - new Tuple3<>("B", 1.0, "Z")); - - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - Table table = tableEnv - .fromDataSet(data, "groupMe, value, name") - .select("groupMe, value, name") - .where("groupMe != 'B'"); - - DataSet myPojos = tableEnv.toDataSet(table, MyPojo.class); - - DataSet result = myPojos.groupBy("groupMe") - .sortGroup("value", Order.DESCENDING) - .first(1); - - List resultList = result.collect(); - compareResultAsText(resultList, "A,24.0,Y"); - } - - public static class MyPojo implements Serializable { - private static final long serialVersionUID = 8741918940120107213L; - - public String groupMe; - public double value; - public String name; - - public MyPojo() { - // for serialization - } - - public MyPojo(String groupMe, double value, String name) { - this.groupMe = groupMe; - this.value = value; - this.name = name; - } - - @Override - public String toString() { - return groupMe + "," + value + "," + name; - } - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala index 7c0cdff128958..16c8ececdb982 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala @@ -19,12 +19,14 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.examples.scala.WordCountTable.{WC => MyWC} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.test.util.TestBaseUtils import org.junit._ import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -32,13 +34,16 @@ import org.junit.runners.Parameterized import scala.collection.JavaConverters._ @RunWith(classOf[Parameterized]) -class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { +class AggregationsITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsTestBase(mode, configMode) { @Test def testAggregationTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) .select('_1.sum, '_1.min, '_1.max, '_1.count, '_1.avg) @@ -52,7 +57,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testAggregationOnNonExistingField(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv) // Must fail. Field 'foo does not exist. @@ -63,7 +68,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testWorkingAggregationDataTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements( (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"), @@ -79,7 +84,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testProjection(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements( (1: Byte, 1: Short), @@ -95,7 +100,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testAggregationWithArithmetic(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv) .select(('_1 + 2).avg + 2, '_2.count + 5) @@ -109,7 +114,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testAggregationWithTwoCount(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv) .select('_1.count, '_2.count) @@ -123,7 +128,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testAggregationAfterProjection(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements( (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"), @@ -140,7 +145,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testNonWorkingAggregationDataTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements(("Hello", 1)).toTable(tEnv) // Must fail. Field '_1 is not a numeric type. @@ -153,7 +158,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testNoNestedAggregations(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements(("Hello", 1)).toTable(tEnv) // Must fail. Sum aggregation can not be chained. @@ -164,7 +169,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testSQLStyleAggregations(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) .select( @@ -184,7 +189,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa def testPojoAggregation(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val input = env.fromElements( MyWC("hello", 1), @@ -204,5 +209,196 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa TestBaseUtils.compareResultAsText(mappedResult.asJava, expected) } + @Test + def testDistinct(): Unit = { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + val distinct = ds.select('b).distinct() + + val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n" + val results = distinct.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testDistinctAfterAggregate(): Unit = { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e) + val distinct = ds.groupBy('a, 'e).select('e).distinct() + + val expected = "1\n" + "2\n" + "3\n" + val results = distinct.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test(expected = classOf[ValidationException]) + def testGroupingOnNonExistentField(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + // must fail. '_foo not a valid field + .groupBy('_foo) + .select('a.avg) + } + + @Test(expected = classOf[ValidationException]) + def testGroupingInvalidSelection(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .groupBy('a, 'b) + // must fail. 'c is not a grouping key or aggregation + .select('c) + } + + @Test + def testGroupedAggregate(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('b, 'a.sum) + + val expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupingKeyForwardIfNotUsed(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('a.sum) + + val expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupNoAggregation(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env) + .toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('a.sum as 'd, 'b) + .groupBy('b, 'd) + .select('b) + + val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupedAggregateWithLongKeys(): Unit = { + // This uses very long keys to force serialized comparison. + // With short keys, the normalized key is sufficient. + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds = env.fromElements( + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), + ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2)) + .rebalance().setParallelism(2).toTable(tEnv, 'a, 'b, 'c) + .groupBy('a, 'b) + .select('c.sum) + + val expected = "10\n" + "8\n" + val results = ds.collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupedAggregateWithConstant1(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .select('a, 4 as 'four, 'b) + .groupBy('four, 'a) + .select('four, 'b.sum) + + val expected = "4,2\n" + "4,3\n" + "4,5\n" + "4,5\n" + "4,5\n" + "4,6\n" + + "4,6\n" + "4,6\n" + "4,3\n" + "4,4\n" + "4,6\n" + "4,1\n" + "4,4\n" + + "4,4\n" + "4,5\n" + "4,6\n" + "4,2\n" + "4,3\n" + "4,4\n" + "4,5\n" + "4,6\n" + val results = t.toDataSet[Row].collect() + + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupedAggregateWithConstant2(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .select('b, 4 as 'four, 'a) + .groupBy('b, 'four) + .select('four, 'a.sum) + + val expected = "4,1\n" + "4,5\n" + "4,15\n" + "4,34\n" + "4,65\n" + "4,111\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupedAggregateWithExpression(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e) + .groupBy('e, 'b % 3) + .select('c.min, 'e, 'a.avg, 'd.count) + + val expected = "0,1,1,1\n" + "3,2,3,3\n" + "7,1,4,2\n" + "14,2,5,1\n" + + "5,3,4,2\n" + "2,1,3,2\n" + "1,2,3,3\n" + "12,3,5,1" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testGroupedAggregateWithFilter(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('b, 'a.sum) + .where('b === 2) + + val expected = "2,5\n" + val results = t.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/DistinctITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/DistinctITCase.scala deleted file mode 100644 index 55c7944a16cdc..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/DistinctITCase.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.api.scala.batch.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit.Test -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class DistinctITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - - @Test - def testDistinct(): Unit = { - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val distinct = ds.select('b).distinct() - - val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n" - val results = distinct.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testDistinctAfterAggregate(): Unit = { - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val ds = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e) - val distinct = ds.groupBy('a, 'e).select('e).distinct() - - val expected = "1\n" + "2\n" + "3\n" - val results = distinct.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala deleted file mode 100644 index 8889b3766c155..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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.api.scala.batch.table - -import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} -import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} -import org.junit._ -import org.junit.runner.RunWith -import org.junit.runners.Parameterized - -import scala.collection.JavaConverters._ - -@RunWith(classOf[Parameterized]) -class GroupedAggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - - @Test(expected = classOf[ValidationException]) - def testGroupingOnNonExistentField(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - // must fail. '_foo not a valid field - .groupBy('_foo) - .select('a.avg) - } - - @Test(expected = classOf[ValidationException]) - def testGroupingInvalidSelection(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .groupBy('a, 'b) - // must fail. 'c is not a grouping key or aggregation - .select('c) - } - - @Test - def testGroupedAggregate(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .groupBy('b) - .select('b, 'a.sum) - - val expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupingKeyForwardIfNotUsed(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .groupBy('b) - .select('a.sum) - - val expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupNoAggregation(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env) - .toTable(tEnv, 'a, 'b, 'c) - .groupBy('b) - .select('a.sum as 'd, 'b) - .groupBy('b, 'd) - .select('b) - - val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupedAggregateWithLongKeys(): Unit = { - // This uses very long keys to force serialized comparison. - // With short keys, the normalized key is sufficient. - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val ds = env.fromElements( - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2), - ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2)) - .rebalance().setParallelism(2).toTable(tEnv, 'a, 'b, 'c) - .groupBy('a, 'b) - .select('c.sum) - - val expected = "10\n" + "8\n" - val results = ds.collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupedAggregateWithConstant1(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .select('a, 4 as 'four, 'b) - .groupBy('four, 'a) - .select('four, 'b.sum) - - val expected = "4,2\n" + "4,3\n" + "4,5\n" + "4,5\n" + "4,5\n" + "4,6\n" + - "4,6\n" + "4,6\n" + "4,3\n" + "4,4\n" + "4,6\n" + "4,1\n" + "4,4\n" + - "4,4\n" + "4,5\n" + "4,6\n" + "4,2\n" + "4,3\n" + "4,4\n" + "4,5\n" + "4,6\n" - val results = t.toDataSet[Row].collect() - - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupedAggregateWithConstant2(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .select('b, 4 as 'four, 'a) - .groupBy('b, 'four) - .select('four, 'a.sum) - - val expected = "4,1\n" + "4,5\n" + "4,15\n" + "4,34\n" + "4,65\n" + "4,111\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupedAggregateWithExpression(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e) - .groupBy('e, 'b % 3) - .select('c.min, 'e, 'a.avg, 'd.count) - - val expected = "0,1,1,1\n" + "3,2,3,3\n" + "7,1,4,2\n" + "14,2,5,1\n" + - "5,3,4,2\n" + "2,1,3,2\n" + "1,2,3,3\n" + "12,3,5,1" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testGroupedAggregateWithFilter(): Unit = { - - val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) - - val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - .groupBy('b) - .select('b, 'a.sum) - .where('b === 2) - - val expected = "2,5\n" - val results = t.toDataSet[Row].collect() - TestBaseUtils.compareResultAsText(results.asJava, expected) - } -} From 0d9bf84e33a5b5ab59631f113024b58e2ff36a9e Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 29 Sep 2016 11:29:49 +0200 Subject: [PATCH 4/5] All batch ITCases use TableProgramsTestBase --- .../api/java/batch/table/JoinITCase.java | 7 ++- .../api/scala/batch/TableSinkITCase.scala | 14 +++-- .../api/scala/batch/TableSourceITCase.scala | 12 ++-- .../api/scala/batch/table/JoinITCase.scala | 55 ++++++++++--------- 4 files changed, 49 insertions(+), 39 deletions(-) diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java index e6db3b0010146..22257b06b8349 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.batch.table; +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.java.DataSet; @@ -37,10 +38,10 @@ @RunWith(Parameterized.class) -public class JoinITCase extends MultipleProgramsTestBase { +public class JoinITCase extends TableProgramsTestBase { - public JoinITCase(TestExecutionMode mode) { - super(mode); + public JoinITCase(TestExecutionMode mode, TableConfigMode configMode){ + super(mode, configMode); } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala index 407fa4cefb9ae..d7e99d4e6627d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala @@ -20,14 +20,15 @@ package org.apache.flink.api.scala.batch import java.io.File -import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.{ExecutionEnvironment, _} +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ -import org.apache.flink.api.scala.ExecutionEnvironment import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.TableEnvironment import org.apache.flink.api.table.sinks.CsvTableSink -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode +import org.apache.flink.test.util.TestBaseUtils import org.junit.Test import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -35,8 +36,9 @@ import org.junit.runners.Parameterized @RunWith(classOf[Parameterized]) class TableSinkITCase( - mode: TestExecutionMode) - extends MultipleProgramsTestBase(mode) { + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsTestBase(mode, configMode) { @Test def testBatchTableSink(): Unit = { @@ -46,7 +48,7 @@ class TableSinkITCase( val path = tmpFile.toURI.toString val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) env.setParallelism(4) val input = CollectionDataSets.get3TupleDataSet(env) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala index 6fd0d134e81c9..08bee72601a9d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala @@ -39,14 +39,16 @@ import org.junit.runners.Parameterized import scala.collection.JavaConverters._ @RunWith(classOf[Parameterized]) -class TableSourceITCase(mode: TestExecutionMode) - extends MultipleProgramsTestBase(mode) { +class TableSourceITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsTestBase(mode, configMode) { @Test def testBatchTableSourceTableAPI(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.registerTableSource("MyTestTable", new TestBatchTableSource()) val results = tEnv @@ -65,7 +67,7 @@ class TableSourceITCase(mode: TestExecutionMode) def testBatchTableSourceSQL(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.registerTableSource("MyTestTable", new TestBatchTableSource()) val results = tEnv.sql( @@ -100,7 +102,7 @@ class TableSourceITCase(mode: TestExecutionMode) tmpWriter.close() val env = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val csvTable = new CsvTableSource( tempFile.getAbsolutePath, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala index f6e6081b141bd..67cac14d655e3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala @@ -19,12 +19,14 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{TableException, ValidationException, Row, TableEnvironment} import org.apache.flink.api.table.expressions.Literal +import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.test.util.TestBaseUtils import org.junit._ import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -32,12 +34,15 @@ import org.junit.runners.Parameterized import scala.collection.JavaConverters._ @RunWith(classOf[Parameterized]) -class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { +class JoinITCase( + mode: TestExecutionMode, + configMode: TableConfigMode) + extends TableProgramsTestBase(mode, configMode) { @Test def testJoin(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -53,7 +58,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) def testJoinWithFilter(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h) @@ -68,7 +73,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithJoinFilter(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -84,7 +89,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithMultipleKeys(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -100,7 +105,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testJoinNonExistingKey(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -114,7 +119,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testJoinWithNonMatchingKeyTypes(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -128,7 +133,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testJoinWithAmbiguousFields(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'c) @@ -142,7 +147,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[TableException]) def testNoEqualityJoinPredicate1(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -156,7 +161,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[TableException]) def testNoEqualityJoinPredicate2(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -170,7 +175,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithAggregation(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -185,7 +190,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithGroupedAggregation(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -203,7 +208,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinPushThroughJoin(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -223,7 +228,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithDisjunctivePred(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -240,7 +245,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testJoinWithExpressionPreds(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) @@ -259,8 +264,8 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testJoinTablesFromDifferentEnvs(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv1 = TableEnvironment.getTableEnvironment(env) - val tEnv2 = TableEnvironment.getTableEnvironment(env) + val tEnv1 = TableEnvironment.getTableEnvironment(env, config) + val tEnv2 = TableEnvironment.getTableEnvironment(env, config) val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv2, 'd, 'e, 'f, 'g, 'h) @@ -272,7 +277,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testLeftJoinWithMultipleKeys(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) @@ -294,7 +299,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testNoJoinCondition(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) @@ -306,7 +311,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test(expected = classOf[ValidationException]) def testNoEquiJoin(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) @@ -318,7 +323,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testRightJoinWithMultipleKeys(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) @@ -337,7 +342,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testRightJoinWithNotOnlyEquiJoin(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) @@ -353,7 +358,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) @Test def testFullOuterJoinWithMultipleKeys(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env, config) tEnv.getConfig.setNullCheck(true) val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) From 838371886e62e55b240da32b47ce93e69145d345 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 29 Sep 2016 15:26:57 +0200 Subject: [PATCH 5/5] fixes --- .../api/java/batch/TableEnvironmentITCase.java | 1 - .../java/batch/table/AggregationsITCase.java | 18 +++++++----------- .../flink/api/java/batch/table/CalcITCase.java | 10 ++++++++++ .../api/java/batch/table/CastingITCase.java | 2 -- .../flink/api/java/batch/table/JoinITCase.java | 10 ++++------ .../scala/batch/TableEnvironmentITCase.scala | 5 ++--- .../flink/api/scala/batch/sql/CalcITCase.scala | 12 ++++++++++++ .../batch/utils/TableProgramsTestBase.scala | 8 ++++---- 8 files changed, 39 insertions(+), 27 deletions(-) diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java index da66387db6816..5e40724f99be3 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java @@ -54,7 +54,6 @@ public TableEnvironmentITCase(TestExecutionMode mode, TableConfigMode configMode public static Collection parameters() { return Arrays.asList(new Object[][] { { TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() }, - { TestExecutionMode.COLLECTION, TableProgramsTestBase.NULL() }, { TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT() } }); } diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java index 2b42b95554ab3..02f6e0b1ab04c 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java @@ -18,31 +18,27 @@ package org.apache.flink.api.java.batch.table; import java.io.Serializable; -import java.util.Arrays; -import java.util.Collection; +import java.util.List; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.operators.DataSource; +import org.apache.flink.api.java.table.BatchTableEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.ValidationException; +import org.apache.flink.examples.java.WordCountTable.WC; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import org.apache.flink.examples.java.WordCountTable.WC; - -import java.util.List; @RunWith(Parameterized.class) public class AggregationsITCase extends TableProgramsTestBase { diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java index 3f74b7155c9fc..fcdf2e19003ec 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java @@ -18,6 +18,8 @@ package org.apache.flink.api.java.batch.table; +import java.util.Arrays; +import java.util.Collection; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.Row; @@ -41,6 +43,14 @@ public CalcITCase(TestExecutionMode mode, TableConfigMode configMode){ super(mode, configMode); } + @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}") + public static Collection parameters() { + return Arrays.asList(new Object[][] { + { TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() }, + { TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL() } + }); + } + @Test public void testSimpleSelectAllWithAs() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java index 9646076650cf0..333953b414277 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.operators.DataSource; import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.api.java.tuple.Tuple6; @@ -32,7 +31,6 @@ import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java index 22257b06b8349..9676608bc6014 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java @@ -18,24 +18,22 @@ package org.apache.flink.api.java.batch.table; -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.Table; +import java.util.List; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; +import org.apache.flink.api.table.Row; +import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.List; - @RunWith(Parameterized.class) public class JoinITCase extends TableProgramsTestBase { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala index 36b2969603bbe..2d82dbcc51247 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala @@ -143,7 +143,7 @@ class TableEnvironmentITCase( tEnv2.registerTable("MyTable", t1) } - @Test + @Test def testToTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -263,8 +263,7 @@ object TableEnvironmentITCase { def parameters(): util.Collection[Array[java.lang.Object]] = { Seq[Array[AnyRef]]( Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT), - Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT), - Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT)).asJava + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT)).asJava } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala index 56045214a79f0..49a97e33469a8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala @@ -18,6 +18,8 @@ package org.apache.flink.api.scala.batch.sql +import java.util + import org.apache.flink.api.scala._ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode @@ -263,3 +265,13 @@ class CalcITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } } + +object CalcITCase { + + @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}") + def parameters(): util.Collection[Array[java.lang.Object]] = { + Seq[Array[AnyRef]]( + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT), + Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL)).asJava + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala index becfade8dc8b4..2ce42d439ea42 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.scala.batch.utils import java.util -import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.{EFFICIENT, NULL, TableConfigMode} +import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.{EFFICIENT, NO_NULL, TableConfigMode} import org.apache.flink.api.table.TableConfig import org.apache.flink.test.util.MultipleProgramsTestBase import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode @@ -36,7 +36,7 @@ class TableProgramsTestBase( def config: TableConfig = { val conf = new TableConfig tableConfigMode match { - case NULL => + case NO_NULL => conf.setNullCheck(false) case EFFICIENT => conf.setEfficientTypeUsage(true) @@ -49,8 +49,8 @@ class TableProgramsTestBase( object TableProgramsTestBase { case class TableConfigMode(nullCheck: Boolean, efficientTypes: Boolean) - val DEFAULT = TableConfigMode(nullCheck = false, efficientTypes = false) - val NULL = TableConfigMode(nullCheck = true, efficientTypes = false) + val DEFAULT = TableConfigMode(nullCheck = true, efficientTypes = false) + val NO_NULL = TableConfigMode(nullCheck = false, efficientTypes = false) val EFFICIENT = TableConfigMode(nullCheck = false, efficientTypes = true) @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")