You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ch...@apache.org on 2013/04/18 18:19:20 UTC

svn commit: r1469407 - in /pig/trunk: ./ src/org/apache/pig/parser/ test/ test/org/apache/pig/test/

Author: cheolsoo
Date: Thu Apr 18 16:19:19 2013
New Revision: 1469407

URL: http://svn.apache.org/r1469407
Log:
PIG-3268: Case statement support (cheolsoo)

Added:
    pig/trunk/test/org/apache/pig/test/TestCase.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/parser/AliasMasker.g
    pig/trunk/src/org/apache/pig/parser/AstPrinter.g
    pig/trunk/src/org/apache/pig/parser/AstValidator.g
    pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
    pig/trunk/src/org/apache/pig/parser/QueryLexer.g
    pig/trunk/src/org/apache/pig/parser/QueryParser.g
    pig/trunk/test/commit-tests

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Apr 18 16:19:19 2013
@@ -28,6 +28,8 @@ PIG-3174:  Remove rpm and deb artifacts 
 
 IMPROVEMENTS
 
+PIG-3268: Case statement support (cheolsoo)
+
 PIG-3269: In operator support (cheolsoo)
 
 PIG-200: Pig Performance Benchmarks (daijy)

Modified: pig/trunk/src/org/apache/pig/parser/AliasMasker.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/AliasMasker.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/AliasMasker.g (original)
+++ pig/trunk/src/org/apache/pig/parser/AliasMasker.g Thu Apr 18 16:19:19 2013
@@ -309,10 +309,15 @@ cond
     | ^( NOT cond )
     | ^( NULL expr NOT? )
     | ^( rel_op expr expr )
+    | in_eval
     | func_eval
     | ^( BOOL_COND expr )
 ;
 
+in_eval
+    : ^( IN expr expr+ )
+;
+
 func_eval
     : ^( FUNC_EVAL func_name real_arg* )
 ;
@@ -352,7 +357,7 @@ var_expr
 ;
 
 projectable_expr
-    : func_eval | col_ref | bin_expr
+    : func_eval | col_ref | bin_expr | case_expr
 ;
 
 dot_proj
@@ -383,6 +388,10 @@ bin_expr
     : ^( BIN_EXPR cond expr expr )
 ;
 
+case_expr
+    : ^( CASE expr+ )
+;
+
 limit_clause
     : ^( LIMIT rel ( INTEGER | LONGINTEGER | expr ) )
 ;

Modified: pig/trunk/src/org/apache/pig/parser/AstPrinter.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/AstPrinter.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/AstPrinter.g (original)
+++ pig/trunk/src/org/apache/pig/parser/AstPrinter.g Thu Apr 18 16:19:19 2013
@@ -331,7 +331,7 @@ var_expr
 ;
 
 projectable_expr
-    : func_eval | col_ref | bin_expr
+    : func_eval | col_ref | bin_expr | case_expr
 ;
 
 dot_proj
@@ -363,6 +363,10 @@ bin_expr
     : ^( BIN_EXPR { sb.append(" ("); } cond { sb.append(" ? "); } expr { sb.append(" : "); } expr { sb.append(") "); } )
 ;
 
+case_expr
+    : ^( CASE { sb.append(" " + $CASE.text + "("); } expr ( { sb.append(", "); } expr )+ { sb.append(") "); } )
+;
+
 limit_clause
     : ^( LIMIT { sb.append($LIMIT.text).append(" "); } rel
         ( INTEGER { sb.append(" ").append($INTEGER.text); } | LONGINTEGER { sb.append(" ").append($LONGINTEGER.text); } | expr ) )
@@ -680,6 +684,7 @@ eid : rel_str_op
     | TOMAP      { sb.append($TOMAP.text); }
     | TOTUPLE    { sb.append($TOTUPLE.text); }
     | IN         { sb.append($IN.text); }
+    | CASE       { sb.append($CASE.text); }
 ;
 
 // relational operator

Modified: pig/trunk/src/org/apache/pig/parser/AstValidator.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/AstValidator.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/AstValidator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/AstValidator.g Thu Apr 18 16:19:19 2013
@@ -398,7 +398,7 @@ bag_type_cast : ^( BAG_TYPE_CAST tuple_t
 var_expr : projectable_expr ( dot_proj | pound_proj )*
 ;
 
-projectable_expr: func_eval | col_ref | bin_expr
+projectable_expr: func_eval | col_ref | bin_expr | case_expr
 ;
 
 dot_proj : ^( PERIOD col_alias_or_index+ )
@@ -423,6 +423,9 @@ pound_proj : ^( POUND ( QUOTEDSTRING | N
 bin_expr : ^( BIN_EXPR cond expr expr )
 ;
 
+case_expr: ^( CASE expr+ )
+;
+
 limit_clause : ^( LIMIT rel ( INTEGER | LONGINTEGER | expr ) )
 ;
 

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g Thu Apr 18 16:19:19 2013
@@ -776,7 +776,7 @@ cond[LogicalExpressionPlan exprPlan] ret
    }
  | ^( BOOL_COND e1 = expr[$exprPlan] )
    {
-   	   $expr = $e1.expr;
+       $expr = $e1.expr;
        $expr.setLocation( new SourceLocation( (PigParserNode)$BOOL_COND ) );
    }
 ;
@@ -990,6 +990,10 @@ projectable_expr[LogicalExpressionPlan p
    {
        $expr = $bin_expr.expr;
    }
+ | case_expr[$plan]
+   {
+       $expr = $case_expr.expr;
+   }
 ;
 
 dot_proj returns[List<Object> cols]
@@ -1039,6 +1043,40 @@ bin_expr[LogicalExpressionPlan plan] ret
    }
 ;
 
+case_expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+@init {
+    List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+}
+ : ^( CASE ( expr[$plan] { exprs.add($expr.expr); } )+ )
+    {
+        // Convert CASE tree to nested bincond expressions. Please also see
+        // QueryParser.g for how CASE tree is constructed from case statement.
+        boolean hasElse = exprs.size() \% 3 == 1;
+        LogicalExpression elseExpr = hasElse ? exprs.get(exprs.size()-1)
+                                             : new ConstantExpression($plan, null);
+
+        int numWhenBranches = exprs.size() / 3;
+        BinCondExpression prevBinCondExpr = null;
+        BinCondExpression currBinCondExpr = null;
+        for (int i = 0; i < numWhenBranches; i++) {
+            if (i == 0) {
+                currBinCondExpr = new BinCondExpression( $plan,
+                    new EqualExpression( $plan, exprs.get(3*i), exprs.get(3*i+1) ),
+                    exprs.get(3*i+2),
+                    elseExpr );
+            } else {
+                currBinCondExpr = new BinCondExpression( $plan,
+                    new EqualExpression( $plan, exprs.get(3*i), exprs.get(3*i+1) ),
+                    exprs.get(3*i+2),
+                    prevBinCondExpr );
+            }
+            prevBinCondExpr = currBinCondExpr;
+        }
+        $expr = currBinCondExpr;
+        $expr.setLocation( new SourceLocation( (PigParserNode)$case_expr.start ) );
+    }
+;
+
 limit_clause returns[String alias]
 scope GScope;
 @init {

Modified: pig/trunk/src/org/apache/pig/parser/QueryLexer.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryLexer.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryLexer.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryLexer.g Thu Apr 18 16:19:19 2013
@@ -264,6 +264,21 @@ RIGHT : 'RIGHT'
 FULL : 'FULL'
 ;
 
+CASE : 'CASE'
+;
+
+WHEN : 'WHEN'
+;
+
+THEN : 'THEN'
+;
+
+ELSE : 'ELSE'
+;
+
+END : 'END'
+;
+
 STR_OP_EQ : 'EQ'
 ;
 

Modified: pig/trunk/src/org/apache/pig/parser/QueryParser.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParser.g?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParser.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParser.g Thu Apr 18 16:19:19 2013
@@ -97,6 +97,17 @@ private static Log log = LogFactory.getL
 
 private Set<String> memory = new HashSet<String>();
 
+// Make a deep copy of the given node
+private static Tree deepCopy(Tree tree) {
+    Tree copy = tree.dupNode();
+    for (int i = 0; i < tree.getChildCount(); i++) {
+        Tree child = deepCopy(tree.getChild(i));
+        child.setParent(copy);
+        copy.addChild(child);
+    }
+    return copy;
+}
+
 @Override
 protected Object recoverFromMismatchedToken(IntStream input, int ttype, BitSet follow)
 throws RecognitionException {
@@ -673,6 +684,32 @@ cast_expr
         && (tree.getChild(0).getType() != NULL || tree.getChild(0).getChildCount() == 0)) {
             ((CommonTree)tree).token.setType(TUPLE_VAL);
         }
+
+        // For CASE statement, we clone the case expression (1st child of the
+        // returned tree) and insert it before every when expression. For example,
+        //
+        //   CASE e1
+        //     WHEN e2 THEN e3
+        //     WHEN e4 THEN e5
+        //     ELSE e6
+        //   END
+        // =>
+        //   ^( CASE e1, e2, e3, e1, e4, e5, e6 )
+        //
+        // Note that e1 appears twice at index 0 and 3.
+        //
+        // This is needed because in LogicalPlanGenerator.g, we translate this
+        // tree to nested bincond expressions, and we need to construct a new
+        // LogicalExpression object per when branch.
+        if(tree.getType() == CASE) {
+            Tree caseExpr = tree.getChild(0);
+            int childCount = tree.getChildCount();
+            boolean hasElse = childCount \% 2 == 0;
+            int whenBranchCount = ( childCount - (hasElse ? 2 : 1) ) / 2;
+            for(int i = 1; i < whenBranchCount; i++) {
+                tree.insertChild(3*i, deepCopy(caseExpr));
+            }
+        }
     }
           : scalar
           | MINUS cast_expr -> ^( NEG cast_expr )
@@ -683,6 +720,7 @@ cast_expr
           | identifier_plus projection*
           | identifier_plus func_name_suffix? LEFT_PAREN ( real_arg ( COMMA real_arg )* )? RIGHT_PAREN projection* -> ^( FUNC_EVAL identifier_plus func_name_suffix? real_arg* ) projection*
           | func_name_without_columns LEFT_PAREN ( real_arg ( COMMA real_arg )* )? RIGHT_PAREN projection* -> ^( FUNC_EVAL func_name_without_columns real_arg* ) projection*
+          | CASE expr WHEN expr THEN expr ( WHEN expr THEN expr )* ( ELSE expr )? END projection* -> ^( CASE expr+ ) projection*
           | paren_expr
           | curly_expr
           | bracket_expr
@@ -975,5 +1013,9 @@ rel_str_op : STR_OP_EQ
 reserved_identifier_whitelist : RANK
                               | CUBE
                               | IN
+                              | WHEN
+                              | THEN
+                              | ELSE
+                              | END
 ;
 

Modified: pig/trunk/test/commit-tests
URL: http://svn.apache.org/viewvc/pig/trunk/test/commit-tests?rev=1469407&r1=1469406&r2=1469407&view=diff
==============================================================================
--- pig/trunk/test/commit-tests (original)
+++ pig/trunk/test/commit-tests Thu Apr 18 16:19:19 2013
@@ -2,6 +2,7 @@
 **/TestBagFormat.java
 **/TestBoolean.java
 **/TestBuiltin.java
+**/TestCase.java
 **/TestCmdLineParser.java
 **/TestCommit.java
 **/TestConstExpr.java

Added: pig/trunk/test/org/apache/pig/test/TestCase.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestCase.java?rev=1469407&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestCase.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestCase.java Thu Apr 18 16:19:19 2013
@@ -0,0 +1,235 @@
+/*
+ * 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.pig.test;
+
+import static junit.framework.Assert.assertEquals;
+import static org.apache.pig.builtin.mock.Storage.resetData;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.junit.Assert.fail;
+
+import java.util.List;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.junit.Test;
+
+public class TestCase {
+
+    /**
+     * Verify that CASE statement without else branch works correctly.
+     * @throws Exception
+     */
+    @Test
+    public void testNoElseBranch() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5),
+                tuple(6),
+                tuple(7)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE i, (" +
+                "  CASE i % 5" +
+                "    WHEN 0 THEN '5n'" +
+                "    WHEN 1 THEN '5n+1'" +
+                "    WHEN 2 THEN '5n+2'" +
+                "    WHEN 3 THEN '5n+3'" +
+                "  END" +
+                ") AS s;");
+        pigServer.registerQuery("C = FILTER B BY s IS NOT NULL;");
+        pigServer.registerQuery("STORE C INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(6, out.size());
+        assertEquals(tuple(1,"5n+1"), out.get(0));
+        assertEquals(tuple(2,"5n+2"), out.get(1));
+        assertEquals(tuple(3,"5n+3"), out.get(2));
+        assertEquals(tuple(5,"5n"),   out.get(3));
+        assertEquals(tuple(6,"5n+1"), out.get(4));
+        assertEquals(tuple(7,"5n+2"), out.get(5));
+    }
+
+    /**
+     * Verify that CASE statement with else branch works correctly.
+     * @throws Exception
+     */
+    @Test
+    public void testWithElseBranch() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5),
+                tuple(6),
+                tuple(7)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE i, (" +
+                "  CASE i % 5" +
+                "    WHEN 0 THEN '5n'" +
+                "    WHEN 1 THEN '5n+1'" +
+                "    WHEN 2 THEN '5n+2'" +
+                "    WHEN 3 THEN '5n+3'" +
+                "    ELSE        '5n+4'" +
+                "  END" +
+                ");");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(7, out.size());
+        assertEquals(tuple(1,"5n+1"), out.get(0));
+        assertEquals(tuple(2,"5n+2"), out.get(1));
+        assertEquals(tuple(3,"5n+3"), out.get(2));
+        assertEquals(tuple(4,"5n+4"), out.get(3));
+        assertEquals(tuple(5,"5n"),   out.get(4));
+        assertEquals(tuple(6,"5n+1"), out.get(5));
+        assertEquals(tuple(7,"5n+2"), out.get(6));
+    }
+
+    /**
+     * Verify that FrontendException is thrown when case expression is missing.
+     * @throws Exception
+     */
+    @Test(expected = FrontendException.class)
+    public void testMissingCaseExpression() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE (" +
+                "  CASE " + // No case expression
+                "    WHEN 0 THEN '3n'" +
+                "    WHEN 1 THEN '3n+1'" +
+                "    ELSE        '3n+2'" +
+                "  END" +
+                ");");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+        fail("FrontendException must be thrown for invalid case statement");
+    }
+
+    /**
+     * Verify that FrontendException is thrown when when expression is missing.
+     * @throws Exception
+     */
+    @Test(expected = FrontendException.class)
+    public void testMissingWhenExpression() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE (" +
+                "  CASE i % 3" +
+                "    WHEN   THEN '3n'" + // No when expression
+                "    WHEN 1 THEN '3n+1'" +
+                "    ELSE        '3n+2'" +
+                "  END" +
+                ");");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+        fail("FrontendException must be thrown for invalid case statement");
+    }
+
+    /**
+     * Verify that FrontendException is thrown when when expression is missing.
+     * @throws Exception
+     */
+    @Test(expected = FrontendException.class)
+    public void testMissingThenExpression() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE (" +
+                "  CASE i % 3" +
+                "    WHEN 0 THEN " + // No then expression
+                "    WHEN 1 THEN '3n+1'" +
+                "    ELSE        '3n+2'" +
+                "  END" +
+                ");");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+        fail("FrontendException must be thrown for invalid case statement");
+    }
+
+    /**
+     * Verify that FrontendException is thrown when when expression is missing.
+     * @throws Exception
+     */
+    @Test(expected = FrontendException.class)
+    public void testMissingElseExpression() throws Exception {
+        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple(1),
+                tuple(2),
+                tuple(3),
+                tuple(4),
+                tuple(5)
+                );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage() AS (i:int);");
+        pigServer.registerQuery("B = FOREACH A GENERATE (" +
+                "  CASE i % 3" +
+                "    WHEN 0 THEN '3n'" +
+                "    WHEN 1 THEN '3n+1'" +
+                "    ELSE " + // No else expression
+                "  END" +
+                ");");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+        fail("FrontendException must be thrown for invalid case statement");
+    }
+}
+