From 6013d2693a4eb3622dc1cbb7d27829ee5b566713 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Aug 2017 19:24:57 +0800 Subject: [PATCH] [FLINK-7451] [table] Support non-ascii character literals in Table API and SQL --- .../table/calcite/FlinkTypeFactory.scala | 7 ++ .../flink/table/expressions/LiteralTest.scala | 71 +++++++++++++++++++ .../UserDefinedScalarFunctionTest.scala | 10 --- 3 files changed, 78 insertions(+), 10 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.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 dbefe203e9601..959db5a7b68db 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 @@ -18,6 +18,8 @@ package org.apache.flink.table.calcite +import java.nio.charset.Charset + import org.apache.calcite.avatica.util.TimeUnit import org.apache.calcite.jdbc.JavaTypeFactoryImpl import org.apache.calcite.rel.`type`._ @@ -25,6 +27,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 @@ -269,6 +272,10 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp canonize(newType) } + + override def getDefaultCharset: Charset = { + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME) + } } object FlinkTypeFactory { 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 0000000000000..97996dc0020a0 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala @@ -0,0 +1,71 @@ +/* + * 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.Types +import org.apache.flink.table.api.scala._ +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 testNonAsciiLiteral(): Unit = { + testAllApis( + 'f0.like("%测试%"), + "f0.like('%测试%')", + "f0 LIKE '%测试%'", + "true") + + testAllApis( + "Абвгде" + "谢谢", + "'Абвгде' + '谢谢'", + "'Абвгде' || '谢谢'", + "Абвгде谢谢") + } + + @Test + def testDoubleQuote(): Unit = { + val hello = "\"\"" + testAllApis( + Func3(42, hello), + s"Func3(42, '$hello')", + s"Func3(42, '$hello')", + s"42 and $hello") + } + + override def testData: Any = { + val testData = new Row(1) + testData.setField(0, "这是个测试字符串") + testData + } + + override def typeInfo: TypeInformation[Any] = { + new RowTypeInfo( + Types.STRING + ).asInstanceOf[TypeInformation[Any]] + } + + override def functions: Map[String, ScalarFunction] = Map( + "Func3" -> Func3 + ) +} 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 71ff70d1d04ce..9b3407e5c12ef 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 @@ -118,16 +118,6 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "-1") } - @Test - def testDoubleQuoteParameters(): Unit = { - val hello = "\"\"" - testAllApis( - Func3(42, hello), - s"Func3(42, '$hello')", - s"Func3(42, '$hello')", - s"42 and $hello") - } - @Test def testResults(): Unit = { testAllApis(