[FLINK-7759] [table] Add support for field names with Boolean prefix. This closes #4829.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/95f86305 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/95f86305 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/95f86305 Branch: refs/heads/master Commit: 95f863052eeb4e6fedffa24ddd0ef4679f8be5ce Parents: 3a45a79 Author: Xpray <[email protected]> Authored: Mon Oct 16 16:13:26 2017 +0800 Committer: Fabian Hueske <[email protected]> Committed: Thu Oct 19 15:08:31 2017 +0200 ---------------------------------------------------------------------- .../table/expressions/ExpressionParser.scala | 2 +- .../table/expressions/LiteralPrefixTest.scala | 79 ++++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/95f86305/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala index f67fbac..201679b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala @@ -201,7 +201,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { } lazy val atom: PackratParser[Expression] = - ( "(" ~> expression <~ ")" ) | literalExpr | fieldReference + ( "(" ~> expression <~ ")" ) | (fieldReference ||| literalExpr) lazy val over: PackratParser[Expression] = composite ~ OVER ~ fieldReference ^^ { case agg ~ _ ~ windowRef => UnresolvedOverCall(agg, windowRef) http://git-wip-us.apache.org/repos/asf/flink/blob/95f86305/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 new file mode 100644 index 0000000..0f46c4e --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/LiteralPrefixTest.scala @@ -0,0 +1,79 @@ +/* + * 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]] + } +}
