Repository: flink Updated Branches: refs/heads/master 5f00294af -> 9d3471574
[FLINK-7451] [table] Support non-ascii character literals in Table API and SQL This closes #4544. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c32fcbc2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c32fcbc2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c32fcbc2 Branch: refs/heads/master Commit: c32fcbc2673233bb1c16a8b96fb2b7e88a55fdca Parents: 5f00294 Author: Jark Wu <j...@apache.org> Authored: Tue Aug 15 19:24:57 2017 +0800 Committer: twalthr <twal...@apache.org> Committed: Wed Nov 15 14:11:57 2017 +0100 ---------------------------------------------------------------------- .../flink/table/calcite/FlinkTypeFactory.scala | 6 + .../table/expressions/LiteralPrefixTest.scala | 79 ------------- .../flink/table/expressions/LiteralTest.scala | 115 +++++++++++++++++++ .../UserDefinedScalarFunctionTest.scala | 10 -- 4 files changed, 121 insertions(+), 89 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/c32fcbc2/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index 04fab76..7bcdc0f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.calcite import java.util +import java.nio.charset.Charset import org.apache.calcite.avatica.util.TimeUnit import org.apache.calcite.jdbc.JavaTypeFactoryImpl @@ -27,6 +28,7 @@ import org.apache.calcite.sql.SqlIntervalQualifier import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName} import org.apache.calcite.sql.parser.SqlParserPos +import org.apache.calcite.util.ConversionUtil import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo._ import org.apache.flink.api.common.typeutils.CompositeType @@ -301,6 +303,10 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp } } } + + override def getDefaultCharset: Charset = { + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME) + } } object FlinkTypeFactory { http://git-wip-us.apache.org/repos/asf/flink/blob/c32fcbc2/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala deleted file mode 100644 index 0f46c4e..0000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala +++ /dev/null @@ -1,79 +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.table.expressions - -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.api.scala._ -import org.apache.flink.table.api.Types -import org.apache.flink.table.expressions.utils.ExpressionTestBase -import org.apache.flink.types.Row -import org.junit.Test - -class LiteralPrefixTest extends ExpressionTestBase { - - @Test - def testFieldWithBooleanPrefix(): Unit = { - - testTableApi( - 'trUeX, - "trUeX", - "trUeX_value" - ) - - testTableApi( - 'FALSE_A, - "FALSE_A", - "FALSE_A_value" - ) - - testTableApi( - 'FALSE_AB, - "FALSE_AB", - "FALSE_AB_value" - ) - - testTableApi( - true, - "trUe", - "true" - ) - - testTableApi( - false, - "FALSE", - "false" - ) - } - - def testData: Any = { - val testData = new Row(3) - testData.setField(0, "trUeX_value") - testData.setField(1, "FALSE_A_value") - testData.setField(2, "FALSE_AB_value") - testData - } - - def typeInfo: TypeInformation[Any] = { - new RowTypeInfo( - Array(Types.STRING, Types.STRING, Types.STRING).asInstanceOf[Array[TypeInformation[_]]], - Array("trUeX", "FALSE_A", "FALSE_AB") - ).asInstanceOf[TypeInformation[Any]] - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/c32fcbc2/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala new file mode 100644 index 0000000..0a60eae --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.expressions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.Types +import org.apache.flink.table.expressions.utils.{ExpressionTestBase, Func3} +import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.types.Row +import org.junit.Test + +class LiteralTest extends ExpressionTestBase { + + @Test + def testFieldWithBooleanPrefix(): Unit = { + + testTableApi( + 'trUeX, + "trUeX", + "trUeX_value" + ) + + testTableApi( + 'FALSE_A, + "FALSE_A", + "FALSE_A_value" + ) + + testTableApi( + 'FALSE_AB, + "FALSE_AB", + "FALSE_AB_value" + ) + + testTableApi( + true, + "trUe", + "true" + ) + + testTableApi( + false, + "FALSE", + "false" + ) + } + + @Test + def testNonAsciiLiteral(): Unit = { + testAllApis( + 'f4.like("%æµè¯%"), + "f4.like('%æµè¯%')", + "f4 LIKE '%æµè¯%'", + "true") + + testAllApis( + "Ðбвгде" + "谢谢", + "'Ðбвгде' + '谢谢'", + "'Ðбвгде' || '谢谢'", + "Ðбвгде谢谢") + } + + @Test + def testDoubleQuote(): Unit = { + val hello = "\"<hello>\"" + testAllApis( + Func3(42, hello), + s"Func3(42, '$hello')", + s"Func3(42, '$hello')", + s"42 and $hello") + } + + def testData: Any = { + val testData = new Row(4) + testData.setField(0, "trUeX_value") + testData.setField(1, "FALSE_A_value") + testData.setField(2, "FALSE_AB_value") + testData.setField(3, "è¿æ¯ä¸ªæµè¯å符串") + testData + } + + def typeInfo: TypeInformation[Any] = { + new RowTypeInfo( + Array( + Types.STRING, + Types.STRING, + Types.STRING, + Types.STRING + ).asInstanceOf[Array[TypeInformation[_]]], + Array("trUeX", "FALSE_A", "FALSE_AB", "f4") + ).asInstanceOf[TypeInformation[Any]] + } + + override def functions: Map[String, ScalarFunction] = Map( + "Func3" -> Func3 + ) +} http://git-wip-us.apache.org/repos/asf/flink/blob/c32fcbc2/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala index a3b2f07..a01f2ae 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala @@ -137,16 +137,6 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { } @Test - def testDoubleQuoteParameters(): Unit = { - val hello = "\"<hello>\"" - testAllApis( - Func3(42, hello), - s"Func3(42, '$hello')", - s"Func3(42, '$hello')", - s"42 and $hello") - } - - @Test def testResults(): Unit = { testAllApis( Func4(),