You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2018/09/16 21:42:53 UTC

[flink] branch master updated: [FLINK-10222] [table] Fix parsing of keywords.

This is an automated email from the ASF dual-hosted git repository.

fhueske pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 4f219f4  [FLINK-10222] [table] Fix parsing of keywords.
4f219f4 is described below

commit 4f219f40c3ab8c454ce52a60a857c3a2a0c56451
Author: yanghua <ya...@gmail.com>
AuthorDate: Tue Aug 28 17:31:39 2018 +0800

    [FLINK-10222] [table] Fix parsing of keywords.
    
    This closes #6622.
---
 .../flink/table/expressions/ExpressionParser.scala |  4 +-
 .../flink/table/expressions/KeywordParseTest.scala | 67 ++++++++++++++++++++++
 2 files changed, 70 insertions(+), 1 deletion(-)

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 4909d2c..c82d556 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
@@ -41,8 +41,10 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {
   case class Keyword(key: String)
 
   // Convert the keyword into an case insensitive Parser
+  // The pattern ensures that the keyword is not matched as a prefix, i.e.,
+  //   the keyword is not followed by a Java identifier character.
   implicit def keyword2Parser(kw: Keyword): Parser[String] = {
-    ("""(?i)\Q""" + kw.key + """\E""").r
+    ("""(?i)\Q""" + kw.key + """\E(?![_$\p{javaJavaIdentifierPart}])""").r
   }
 
   // Keyword
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/KeywordParseTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/KeywordParseTest.scala
new file mode 100644
index 0000000..0e89c7f
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/KeywordParseTest.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.expressions.utils.ExpressionTestBase
+import org.apache.flink.types.Row
+import org.junit.{Assert, Test}
+
+/**
+  * Tests keyword as suffix.
+  */
+class KeywordParseTest extends ExpressionTestBase {
+
+  @Test
+  def testKeyword(): Unit = {
+    Assert.assertEquals(
+      ExpressionParser.parseExpression("f0.asc"),
+      Asc(UnresolvedFieldReference("f0")))
+    Assert.assertEquals(
+      ExpressionParser.parseExpression("f0.asc()"),
+      Asc(UnresolvedFieldReference("f0")))
+  }
+
+  @Test
+  def testKeywordAsPrefixInFunctionName(): Unit = {
+    Assert.assertEquals(
+      ExpressionParser.parseExpression("f0.ascii()").asInstanceOf[Call].functionName,
+      "ASCII")
+  }
+
+  @Test
+  def testKeywordAsInfixInFunctionName(): Unit = {
+    Assert.assertEquals(
+      ExpressionParser.parseExpression("f0.iiascii()").asInstanceOf[Call].functionName,
+      "IIASCII")
+  }
+
+  @Test
+  def testKeywordAsSuffixInFunctionName(): Unit = {
+    Assert.assertEquals(
+      ExpressionParser.parseExpression("f0.iiasc()").asInstanceOf[Call].functionName,
+      "IIASC")
+  }
+
+  override def testData: Any = new Row(0)
+
+  override def typeInfo: TypeInformation[Any] =
+    new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
+}