Skip to content

Commit

Permalink
[FLINK-7451] [table] Support non-ascii character literals in Table AP…
Browse files Browse the repository at this point in the history
…I and SQL

This closes apache#4544.
  • Loading branch information
wuchong authored and twalthr committed Nov 15, 2017
1 parent 5f00294 commit c32fcbc
Show file tree
Hide file tree
Showing 3 changed files with 47 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -301,6 +303,10 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp
}
}
}

override def getDefaultCharset: Charset = {
Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME)
}
}

object FlinkTypeFactory {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,12 @@ 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.table.expressions.utils.{ExpressionTestBase, Func3}
import org.apache.flink.table.functions.ScalarFunction
import org.apache.flink.types.Row
import org.junit.Test

class LiteralPrefixTest extends ExpressionTestBase {
class LiteralTest extends ExpressionTestBase {

@Test
def testFieldWithBooleanPrefix(): Unit = {
Expand Down Expand Up @@ -62,18 +63,53 @@ class LiteralPrefixTest extends ExpressionTestBase {
)
}

@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(3)
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).asInstanceOf[Array[TypeInformation[_]]],
Array("trUeX", "FALSE_A", "FALSE_AB")
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
)
}
Original file line number Diff line number Diff line change
Expand Up @@ -136,16 +136,6 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase {
"-1")
}

@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(
Expand Down

0 comments on commit c32fcbc

Please sign in to comment.