Repository: flink Updated Branches: refs/heads/master b40cba609 -> bf6df12d8
[FLINK-4109] [tableAPI] Change the name of ternary condition operator 'eval' to '?' This closes #2173. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bf6df12d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bf6df12d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bf6df12d Branch: refs/heads/master Commit: bf6df12d81e721d6a42caf5d9682baf230bbabfb Parents: b40cba6 Author: Jark Wu <wuchong...@alibaba-inc.com> Authored: Tue Jun 28 12:37:12 2016 +0800 Committer: twalthr <twal...@apache.org> Committed: Mon Jul 4 17:01:30 2016 +0200 ---------------------------------------------------------------------- docs/apis/table.md | 4 ++-- .../flink/api/scala/table/expressionDsl.scala | 8 ++++---- .../table/expressions/ExpressionParser.scala | 20 ++++++++++---------- .../flink/api/table/expressions/logic.scala | 4 ++-- .../api/java/batch/table/ExpressionsITCase.java | 12 ++++++------ .../scala/batch/table/ExpressionsITCase.scala | 12 ++++++------ 6 files changed, 30 insertions(+), 30 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/docs/apis/table.md ---------------------------------------------------------------------- diff --git a/docs/apis/table.md b/docs/apis/table.md index 3f31bb5..4a42633 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -748,7 +748,7 @@ unary = [ "!" | "-" ] , composite ; composite = suffixed | atom ; -suffixed = cast | as | aggregation | nullCheck | evaluate | functionCall ; +suffixed = cast | as | aggregation | nullCheck | if | functionCall ; cast = composite , ".cast(" , dataType , ")" ; @@ -760,7 +760,7 @@ aggregation = composite , ( ".sum" | ".min" | ".max" | ".count" | ".avg" ) , [ " nullCheck = composite , ( ".isNull" | ".isNotNull" ) , [ "()" ] ; -evaluate = composite , ".eval(" , expression , "," , expression , ")" ; +if = composite , ".?(" , expression , "," , expression , ")" ; functionCall = composite , "." , functionIdentifier , "(" , [ expression , { "," , expression } ] , ")" http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala index 1adf2ed..c4287b6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala @@ -71,16 +71,16 @@ trait ImplicitExpressionOperations { def desc = Desc(expr) /** - * Conditional operator that decides which of two other expressions should be evaluated + * Ternary conditional operator that decides which of two other expressions should be evaluated * based on a evaluated boolean condition. * - * e.g. (42 > 5).eval("A", "B") leads to "A" + * e.g. (42 > 5).?("A", "B") leads to "A" * * @param ifTrue expression to be evaluated if condition holds * @param ifFalse expression to be evaluated if condition does not hold */ - def eval(ifTrue: Expression, ifFalse: Expression) = { - Eval(expr, ifTrue, ifFalse) + def ?(ifTrue: Expression, ifFalse: Expression) = { + If(expr, ifTrue, ifFalse) } // scalar functions http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala index 92b95ed..6aa4f89 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala @@ -51,7 +51,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val IS_NOT_NULL: Keyword = Keyword("isNotNull") lazy val CAST: Keyword = Keyword("cast") lazy val NULL: Keyword = Keyword("Null") - lazy val EVAL: Keyword = Keyword("eval") + lazy val IF: Keyword = Keyword("?") lazy val ASC: Keyword = Keyword("asc") lazy val DESC: Keyword = Keyword("desc") lazy val TO_DATE: Keyword = Keyword("toDate") @@ -61,7 +61,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { def functionIdent: ExpressionParser.Parser[String] = not(AS) ~ not(COUNT) ~ not(AVG) ~ not(MIN) ~ not(MAX) ~ not(SUM) ~ not(IS_NULL) ~ not(IS_NOT_NULL) ~ not(CAST) ~ not(NULL) ~ - not(EVAL) ~> super.ident + not(IF) ~> super.ident // data types @@ -173,9 +173,9 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { Trim(TrimConstants.TRIM_BOTH, TrimConstants.TRIM_DEFAULT_CHAR, e) } - lazy val suffixEval: PackratParser[Expression] = - composite ~ "." ~ EVAL ~ "(" ~ expression ~ "," ~ expression ~ ")" ^^ { - case condition ~ _ ~ _ ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => Eval(condition, ifTrue, ifFalse) + lazy val suffixIf: PackratParser[Expression] = + composite ~ "." ~ IF ~ "(" ~ expression ~ "," ~ expression ~ ")" ^^ { + case condition ~ _ ~ _ ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse) } lazy val suffixFunctionCall = @@ -200,7 +200,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val suffixed: PackratParser[Expression] = suffixIsNull | suffixIsNotNull | suffixSum | suffixMin | suffixMax | suffixCount | suffixAvg | - suffixCast | suffixAs | suffixTrim | suffixTrimWithoutArgs | suffixEval | suffixFunctionCall | + suffixCast | suffixAs | suffixTrim | suffixTrimWithoutArgs | suffixIf | suffixFunctionCall | suffixAsc | suffixDesc | suffixToDate | suffixToTimestamp | suffixToTime // prefix operators @@ -236,9 +236,9 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { case _ ~ _ ~ e ~ _ ~ target ~ _ => Alias(e, target.name) } - lazy val prefixEval: PackratParser[Expression] = composite ~ - EVAL ~ "(" ~ expression ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ { - case _ ~ _ ~ condition ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => Eval(condition, ifTrue, ifFalse) + lazy val prefixIf: PackratParser[Expression] = composite ~ + IF ~ "(" ~ expression ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ { + case _ ~ _ ~ condition ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse) } lazy val prefixFunctionCall = functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ { @@ -263,7 +263,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val prefixed: PackratParser[Expression] = prefixIsNull | prefixIsNotNull | prefixSum | prefixMin | prefixMax | prefixCount | prefixAvg | - prefixCast | prefixAs | prefixTrim | prefixTrimWithoutArgs | prefixEval | prefixFunctionCall + prefixCast | prefixAs | prefixTrim | prefixTrimWithoutArgs | prefixIf | prefixFunctionCall // suffix/prefix composite http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala index 90d3dbc..332a8c5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala @@ -76,7 +76,7 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { } } -case class Eval( +case class If( condition: Expression, ifTrue: Expression, ifFalse: Expression) @@ -100,7 +100,7 @@ case class Eval( ValidationSuccess } else { ValidationFailure( - s"Eval should have boolean condition and same type of ifTrue and ifFalse, get " + + s"If should have boolean condition and same type of ifTrue and ifFalse, get " + s"(${condition.resultType}, ${ifTrue.resultType}, ${ifFalse.resultType})") } } http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java index 83c7cfd..7d225f3 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java @@ -135,7 +135,7 @@ public class ExpressionsITCase extends TableProgramsTestBase { } @Test - public void testEval() throws Exception { + public void testIf() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -146,9 +146,9 @@ public class ExpressionsITCase extends TableProgramsTestBase { tableEnv.fromDataSet(input, "a, b"); Table result = table.select( - "(b && true).eval('true', 'false')," + - "false.eval('true', 'false')," + - "true.eval(true.eval(true.eval(10, 4), 4), 4)"); + "(b && true).?('true', 'false')," + + "false.?('true', 'false')," + + "true.?(true.?(true.?(10, 4), 4), 4)"); DataSet<Row> ds = tableEnv.toDataSet(result, Row.class); List<Row> results = ds.collect(); @@ -157,7 +157,7 @@ public class ExpressionsITCase extends TableProgramsTestBase { } @Test(expected = ValidationException.class) - public void testEvalInvalidTypes() throws Exception { + public void testIfInvalidTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -167,7 +167,7 @@ public class ExpressionsITCase extends TableProgramsTestBase { Table table = tableEnv.fromDataSet(input, "a, b"); - Table result = table.select("(b && true).eval(5, 'false')"); + Table result = table.select("(b && true).?(5, 'false')"); DataSet<Row> ds = tableEnv.toDataSet(result, Row.class); List<Row> results = ds.collect(); http://git-wip-us.apache.org/repos/asf/flink/blob/bf6df12d/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala index 3cf3b26..3c3719a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala @@ -129,15 +129,15 @@ class ExpressionsITCase( } @Test - def testEval(): Unit = { + def testIf(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements((5, true)).toTable(tEnv, 'a, 'b) .select( - ('b && true).eval("true", "false"), - false.eval("true", "false"), - true.eval(true.eval(true.eval(10, 4), 4), 4)) + ('b && true).?("true", "false"), + false.?("true", "false"), + true.?(true.?(true.?(10, 4), 4), 4)) val expected = "true,false,10" val results = t.toDataSet[Row].collect() @@ -145,12 +145,12 @@ class ExpressionsITCase( } @Test(expected = classOf[ValidationException]) - def testEvalInvalidTypes(): Unit = { + def testIfInvalidTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val t = env.fromElements((5, true)).toTable(tEnv, 'a, 'b) - .select(('b && true).eval(5, "false")) + .select(('b && true).?(5, "false")) val expected = "true,false,3,10" val results = t.toDataSet[Row].collect()