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/09/21 01:07:56 UTC

svn commit: r1525170 - in /pig/branches/tez: ./ src/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/builtin/ src/org/apache/pig/impl/util/ src/...

Author: cheolsoo
Date: Fri Sep 20 23:07:55 2013
New Revision: 1525170

URL: http://svn.apache.org/r1525170
Log:
Merged latest trunk changes to tez branch

Added:
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
      - copied unchanged from r1525169, pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecutionEngine.java
      - copied unchanged from r1525169, pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecutionEngine.java
    pig/branches/tez/src/org/apache/pig/builtin/Assert.java
      - copied unchanged from r1525169, pig/trunk/src/org/apache/pig/builtin/Assert.java
    pig/branches/tez/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java
      - copied unchanged from r1525169, pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalPlanData.java
    pig/branches/tez/test/org/apache/pig/test/TestAssert.java
      - copied unchanged from r1525169, pig/trunk/test/org/apache/pig/test/TestAssert.java
Removed:
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/MRExecutionEngine.java
Modified:
    pig/branches/tez/   (props changed)
    pig/branches/tez/CHANGES.txt
    pig/branches/tez/src/org/apache/pig/PigServer.java
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalExecType.java
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecType.java
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
    pig/branches/tez/src/org/apache/pig/impl/util/Utils.java
    pig/branches/tez/src/org/apache/pig/parser/AstPrinter.g
    pig/branches/tez/src/org/apache/pig/parser/AstValidator.g
    pig/branches/tez/src/org/apache/pig/parser/LogicalPlanBuilder.java
    pig/branches/tez/src/org/apache/pig/parser/LogicalPlanGenerator.g
    pig/branches/tez/src/org/apache/pig/parser/QueryLexer.g
    pig/branches/tez/src/org/apache/pig/parser/QueryParser.g
    pig/branches/tez/src/org/apache/pig/pen/ExampleGenerator.java
    pig/branches/tez/src/pig-default.properties   (props changed)
    pig/branches/tez/test/org/apache/pig/test/TestLogicalPlanBuilder.java
    pig/branches/tez/test/org/apache/pig/test/TestMRExecutionEngine.java
    pig/branches/tez/test/org/apache/pig/test/TestMultiQueryCompiler.java
    pig/branches/tez/test/org/apache/pig/test/TestMultiQueryLocal.java
    pig/branches/tez/test/org/apache/pig/test/TestPigStats.java
    pig/branches/tez/test/org/apache/pig/test/Util.java
    pig/branches/tez/test/org/apache/pig/test/data/bzipdir1.bz2/bzipdir2.bz2/recordLossblockHeaderEndsAt136500.txt.bz2   (props changed)

Propchange: pig/branches/tez/
------------------------------------------------------------------------------
  Merged /pig/trunk:r1525017-1525169

Modified: pig/branches/tez/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/branches/tez/CHANGES.txt?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/CHANGES.txt (original)
+++ pig/branches/tez/CHANGES.txt Fri Sep 20 23:07:55 2013
@@ -30,6 +30,10 @@ PIG-3174: Remove rpm and deb artifacts f
 
 IMPROVEMENTS
 
+PIG-3199: Provide a method to retriever name of loader/storer in PigServer (prkommireddi via daijy)
+
+PIG-3367: Add assert keyword (operator) in pig (aniket486)
+
 PIG-3235: Avoid extra byte array copies in streaming (rohini)
 
 PIG-3065: pig output format/committer should support recovery for hadoop 0.23 (daijy)
@@ -230,6 +234,8 @@ PIG-3013: BinInterSedes improve chararra
 
 BUG FIXES
 
+PIG-3471: Add a base abstract class for ExecutionEngine (cheolsoo)
+
 PIG-3457: Provide backward compatibility for PigStatsUtil and JobStats (daijy)
 
 PIG-3466: Race Conditions in InternalDistinctBag during proactive spill (cheolsoo)

Modified: pig/branches/tez/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/PigServer.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/PigServer.java (original)
+++ pig/branches/tez/src/org/apache/pig/PigServer.java Fri Sep 20 23:07:55 2013
@@ -82,6 +82,7 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LOLoad;
 import org.apache.pig.newplan.logical.relational.LOStore;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalPlanData;
 import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.newplan.logical.visitor.CastLineageSetter;
@@ -1357,6 +1358,18 @@ public class PigServer {
         }
         return op;
     }
+    
+    /**
+     * Returns data associated with LogicalPlan. It makes
+     * sense to call this method only after a query/script
+     * has been registered with one of the {@link #registerQuery(String)}
+     * or {@link #registerScript(InputStream)} methods.
+     * 
+     * @return LogicalPlanData
+     */
+    public LogicalPlanData getLogicalPlanData() {
+	return new LogicalPlanData(currDAG.lp);
+    }
 
     /*
      * This class holds the internal states of a grunt shell session.

Modified: pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalExecType.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalExecType.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalExecType.java (original)
+++ pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/LocalExecType.java Fri Sep 20 23:07:55 2013
@@ -21,7 +21,6 @@ package org.apache.pig.backend.hadoop.ex
 import java.util.Properties;
 import org.apache.pig.ExecType;
 import org.apache.pig.backend.executionengine.ExecutionEngine;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
 import org.apache.pig.impl.PigContext;
 
 /**

Modified: pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecType.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecType.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecType.java (original)
+++ pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRExecType.java Fri Sep 20 23:07:55 2013
@@ -22,7 +22,6 @@ import java.util.Properties;
 
 import org.apache.pig.ExecType;
 import org.apache.pig.backend.executionengine.ExecutionEngine;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
 import org.apache.pig.impl.PigContext;
 
 /**

Modified: pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java (original)
+++ pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java Fri Sep 20 23:07:55 2013
@@ -50,7 +50,6 @@ import org.apache.pig.backend.BackendExc
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.Launcher;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.LastInputStreamingOptimizer;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.DotMRPrinter;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.EndOfAllInputSetter;

Modified: pig/branches/tez/src/org/apache/pig/impl/util/Utils.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/impl/util/Utils.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/impl/util/Utils.java (original)
+++ pig/branches/tez/src/org/apache/pig/impl/util/Utils.java Fri Sep 20 23:07:55 2013
@@ -23,9 +23,15 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.SequenceInputStream;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.SocketImplFactory;
 import java.util.Collection;
+import java.util.Enumeration;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.commons.logging.Log;
@@ -34,6 +40,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.pig.FileInputLoadFunc;
 import org.apache.pig.FuncSpec;
@@ -351,4 +358,60 @@ public class Utils {
         }
     }
 
+    /**
+     * Method to set the customized SSH socket factory.
+     *
+     * @param pc Pig context that stores the SSH gateway address.
+     */
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    public static void setSSHFactory(PigContext pc) {
+        Properties properties = pc.getProperties();
+        String g = properties.getProperty("ssh.gateway");
+        if (g == null || g.length() == 0) {
+            return;
+        }
+        try {
+            Class clazz = Class.forName("org.apache.pig.shock.SSHSocketImplFactory");
+            SocketImplFactory f = (SocketImplFactory) clazz.getMethod(
+                        "getFactory", new Class[0]).invoke(0, new Object[0]);
+            Socket.setSocketImplFactory(f);
+        } catch (SocketException e) {
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Method to apply pig properties to JobConf (replaces properties with
+     * resulting jobConf values).
+     *
+     * @param conf JobConf with appropriate hadoop resource files
+     * @param properties Pig properties that will override hadoop properties;
+     * properties might be modified
+     */
+    public static void recomputeProperties(JobConf jobConf, Properties properties) {
+        // We need to load the properties from the hadoop configuration
+        // We want to override these with any existing properties we have.
+        if (jobConf != null && properties != null) {
+            // set user properties on the jobConf to ensure that defaults
+            // and deprecation is applied correctly
+            Enumeration<Object> propertiesIter = properties.keys();
+            while (propertiesIter.hasMoreElements()) {
+                String key = (String) propertiesIter.nextElement();
+                String val = properties.getProperty(key);
+                // We do not put user.name, See PIG-1419
+                if (!key.equals("user.name")) {
+                    jobConf.set(key, val);
+                }
+            }
+            // clear user defined properties and re-populate
+            properties.clear();
+            Iterator<Map.Entry<String, String>> iter = jobConf.iterator();
+            while (iter.hasNext()) {
+                Map.Entry<String, String> entry = iter.next();
+                properties.put(entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
 }

Modified: pig/branches/tez/src/org/apache/pig/parser/AstPrinter.g
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/AstPrinter.g?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/AstPrinter.g (original)
+++ pig/branches/tez/src/org/apache/pig/parser/AstPrinter.g Fri Sep 20 23:07:55 2013
@@ -124,6 +124,7 @@ op_clause : define_clause
           | split_clause
           | foreach_clause
           | cube_clause
+          | assert_clause
 ;
 
 define_clause
@@ -302,6 +303,14 @@ store_clause
     : ^( STORE { sb.append($STORE.text).append(" "); } rel { sb.append(" INTO "); } filename ( { sb.append(" USING "); } func_clause)? )
 ;
 
+comment
+    : QUOTEDSTRING { sb.append($QUOTEDSTRING.text); }
+;
+
+assert_clause
+    : ^( ASSERT { sb.append($ASSERT.text).append(" "); } rel {sb.append(" BY ("); } cond { sb.append(")"); } ( { sb.append(" comment: "); } comment)?  )
+;
+
 filter_clause
     : ^( FILTER { sb.append($FILTER.text).append(" "); } rel { sb.append(" BY ("); } cond { sb.append(")"); } )
 ;

Modified: pig/branches/tez/src/org/apache/pig/parser/AstValidator.g
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/AstValidator.g?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/AstValidator.g (original)
+++ pig/branches/tez/src/org/apache/pig/parser/AstValidator.g Fri Sep 20 23:07:55 2013
@@ -174,6 +174,7 @@ op_clause : define_clause
           | split_clause
           | foreach_clause
           | cube_clause
+          | assert_clause
 ;
 
 define_clause : ^( DEFINE alias ( cmd | func_clause ) )
@@ -355,6 +356,12 @@ flatten_clause : ^( FLATTEN expr )
 store_clause : ^( STORE rel filename func_clause? )
 ;
 
+assert_clause : ^( ASSERT rel cond comment? )
+; 
+
+comment : QUOTEDSTRING
+;
+
 filter_clause : ^( FILTER rel cond )
 ;
 

Modified: pig/branches/tez/src/org/apache/pig/parser/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/LogicalPlanBuilder.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/LogicalPlanBuilder.java (original)
+++ pig/branches/tez/src/org/apache/pig/parser/LogicalPlanBuilder.java Fri Sep 20 23:07:55 2013
@@ -38,6 +38,7 @@ import org.apache.pig.PigConfiguration;
 import org.apache.pig.StoreFuncInterface;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.builtin.Assert;
 import org.apache.pig.builtin.CubeDimensions;
 import org.apache.pig.builtin.InvokerGenerator;
 import org.apache.pig.builtin.PigStorage;
@@ -60,6 +61,7 @@ import org.apache.pig.impl.util.MultiMap
 import org.apache.pig.impl.util.StringUtils;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
+import org.apache.pig.newplan.logical.expression.BinCondExpression;
 import org.apache.pig.newplan.logical.expression.ConstantExpression;
 import org.apache.pig.newplan.logical.expression.LessThanExpression;
 import org.apache.pig.newplan.logical.expression.LogicalExpression;
@@ -934,6 +936,29 @@ public class LogicalPlanBuilder {
             throw new ParserValidationException(intStream, loc, ex);
         }
     }
+    
+    String buildAssertOp(SourceLocation loc, LOFilter filterOp, 
+            String alias, String inputAlias, LogicalExpression expr, String comment,
+            LogicalExpressionPlan exprPlan) 
+            throws ParserValidationException {
+        try {
+            filterOp.setAlias(inputAlias);
+            List<LogicalExpression> args = new ArrayList<LogicalExpression>();
+            ConstantExpression lhs = new ConstantExpression(exprPlan, new Boolean(true));
+            ConstantExpression rhs = new ConstantExpression(exprPlan, new Boolean(false));
+            BinCondExpression binCond = new BinCondExpression(exprPlan, expr, lhs, rhs);
+            args.add(binCond);
+            ConstantExpression constExpr = new ConstantExpression(exprPlan, comment);
+            args.add(constExpr);
+            UserFuncExpression udf = new UserFuncExpression(exprPlan, new FuncSpec( Assert.class.getName() ), args );
+            exprPlan.add(udf);
+            filterOp.setFilterPlan(exprPlan);
+            // pass the inputAlias to alias
+            return buildFilterOp(loc, filterOp, inputAlias, inputAlias, exprPlan);
+        } catch (Exception ex) {
+            throw new ParserValidationException(intStream, loc, ex);
+        }
+    }
 
     private String newOperatorKey() {
         return new OperatorKey( scope, getNextId() ).toString();

Modified: pig/branches/tez/src/org/apache/pig/parser/LogicalPlanGenerator.g
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/LogicalPlanGenerator.g?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/LogicalPlanGenerator.g (original)
+++ pig/branches/tez/src/org/apache/pig/parser/LogicalPlanGenerator.g Fri Sep 20 23:07:55 2013
@@ -238,6 +238,7 @@ op_clause returns[String alias] :
           | mr_clause { $alias = $mr_clause.alias; }
           | foreach_clause { $alias = $foreach_clause.alias; }
           | cube_clause { $alias = $cube_clause.alias; }
+          | assert_clause { $alias = $assert_clause.alias; }
 ;
 
 define_clause
@@ -702,6 +703,24 @@ store_clause returns[String alias]
    }
 ;
 
+assert_clause returns[String alias]
+scope GScope;
+@init {
+    $GScope::currentOp = builder.createFilterOp();
+    LogicalExpressionPlan exprPlan = new LogicalExpressionPlan();
+}
+ : ^( ASSERT rel cond[exprPlan] comment? )
+   {
+       SourceLocation loc = new SourceLocation( (PigParserNode)$ASSERT );
+       $alias= builder.buildAssertOp(loc, (LOFilter)$GScope::currentOp, $statement::alias,
+          $statement::inputAlias, $cond.expr, $comment.comment, exprPlan);
+   }
+;
+
+comment returns[String comment]
+ : QUOTEDSTRING { $comment = builder.unquote( $QUOTEDSTRING.text ); }
+;
+
 filter_clause returns[String alias]
 scope GScope;
 @init {

Modified: pig/branches/tez/src/org/apache/pig/parser/QueryLexer.g
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/QueryLexer.g?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/QueryLexer.g (original)
+++ pig/branches/tez/src/org/apache/pig/parser/QueryLexer.g Fri Sep 20 23:07:55 2013
@@ -231,6 +231,9 @@ THROUGH : 'THROUGH'
 STORE : 'STORE'
 ;
 
+ASSERT : 'ASSERT'
+;
+
 MAPREDUCE : 'MAPREDUCE'
 ;
 

Modified: pig/branches/tez/src/org/apache/pig/parser/QueryParser.g
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/parser/QueryParser.g?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/parser/QueryParser.g (original)
+++ pig/branches/tez/src/org/apache/pig/parser/QueryParser.g Fri Sep 20 23:07:55 2013
@@ -382,6 +382,7 @@ op_clause : define_clause
           | union_clause
           | stream_clause
           | mr_clause
+          | assert_clause
 ;
 
 ship_clause : SHIP^ LEFT_PAREN! path_list? RIGHT_PAREN!
@@ -477,6 +478,9 @@ previous_rel : ARROBA
 store_clause : STORE^ rel INTO! QUOTEDSTRING ( USING! func_clause )?
 ;
 
+assert_clause : ASSERT^ rel BY! cond ( COMMA! QUOTEDSTRING )?
+;
+
 filter_clause : FILTER^ rel BY! cond
 ;
 
@@ -1071,5 +1075,6 @@ reserved_identifier_whitelist : RANK
                               | THEN
                               | ELSE
                               | END
+                              | ASSERT
 ;
 

Modified: pig/branches/tez/src/org/apache/pig/pen/ExampleGenerator.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/pen/ExampleGenerator.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/pen/ExampleGenerator.java (original)
+++ pig/branches/tez/src/org/apache/pig/pen/ExampleGenerator.java Fri Sep 20 23:07:55 2013
@@ -30,7 +30,7 @@ import org.apache.pig.impl.util.Identity
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;

Propchange: pig/branches/tez/src/pig-default.properties
------------------------------------------------------------------------------
  Merged /pig/trunk/src/pig-default.properties:r1525017-1525169

Modified: pig/branches/tez/test/org/apache/pig/test/TestLogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestLogicalPlanBuilder.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestLogicalPlanBuilder.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestLogicalPlanBuilder.java Fri Sep 20 23:07:55 2013
@@ -18,6 +18,7 @@
 package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -25,6 +26,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import junit.framework.Assert;
+import junit.framework.AssertionFailedError;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
@@ -59,14 +63,12 @@ import org.apache.pig.newplan.logical.re
 import org.apache.pig.newplan.logical.relational.LOSort;
 import org.apache.pig.newplan.logical.relational.LOStore;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalPlanData;
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
 import org.apache.pig.test.utils.Identity;
 import org.junit.Before;
 import org.junit.Test;
 
-import junit.framework.Assert;
-import junit.framework.AssertionFailedError;
-
 public class TestLogicalPlanBuilder {
     PigContext pigContext = new PigContext(ExecType.LOCAL, new Properties());
     private PigServer pigServer = null;
@@ -2156,6 +2158,21 @@ public class TestLogicalPlanBuilder {
         pigServer.getPigContext().getProperties().remove(PigConfiguration.PIG_DEFAULT_STORE_FUNC);      
     }
     
+    @Test
+    public void testLogicalPlanData() throws Exception {
+        String query = "a = load 'input.txt'; b = load 'anotherinput.txt'; c = join a by $0, b by $1;" +
+                "store c into 'output' using org.apache.pig.test.PigStorageWithSchema();";
+        // Set batch on so the query is not executed
+        pigServer.setBatchOn();
+        pigServer.registerQuery(query);
+        LogicalPlanData lData = pigServer.getLogicalPlanData();
+        assertEquals("LoadFunc must be PigStorage", "org.apache.pig.builtin.PigStorage", lData.getLoadFuncs().get(0));
+        assertEquals("StoreFunc must be PigStorageWithSchema", "org.apache.pig.test.PigStorageWithSchema", lData.getStoreFuncs().get(0));
+        assertEquals("Number of sources must be 2", lData.getSources().size(), 2);
+        assertTrue("Source must end with input.txt", lData.getSources().get(0).endsWith("input.txt"));
+        assertTrue("Sink must end with output", lData.getSinks().get(0).endsWith("output"));
+    }
+    
     /**
      * This method is not generic. Expects logical plan to have atleast
      * 1 source and returns the corresponding FuncSpec.

Modified: pig/branches/tez/test/org/apache/pig/test/TestMRExecutionEngine.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestMRExecutionEngine.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestMRExecutionEngine.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestMRExecutionEngine.java Fri Sep 20 23:07:55 2013
@@ -6,7 +6,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.pig.ExecType;
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.impl.PigContext;
 import org.junit.Test;
 

Modified: pig/branches/tez/test/org/apache/pig/test/TestMultiQueryCompiler.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestMultiQueryCompiler.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestMultiQueryCompiler.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestMultiQueryCompiler.java Fri Sep 20 23:07:55 2013
@@ -30,7 +30,7 @@ import java.util.Iterator;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceOper;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan;

Modified: pig/branches/tez/test/org/apache/pig/test/TestMultiQueryLocal.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestMultiQueryLocal.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestMultiQueryLocal.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestMultiQueryLocal.java Fri Sep 20 23:07:55 2013
@@ -38,7 +38,7 @@ import org.apache.hadoop.mapreduce.TaskA
 import org.apache.pig.ExecType;
 import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigTextOutputFormat;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;

Modified: pig/branches/tez/test/org/apache/pig/test/TestPigStats.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPigStats.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigStats.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigStats.java Fri Sep 20 23:07:55 2013
@@ -36,7 +36,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceOper;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan;

Modified: pig/branches/tez/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/Util.java?rev=1525170&r1=1525169&r2=1525170&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/Util.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/Util.java Fri Sep 20 23:07:55 2013
@@ -65,8 +65,8 @@ import org.apache.pig.PigServer;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
-import org.apache.pig.backend.hadoop.executionengine.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;

Propchange: pig/branches/tez/test/org/apache/pig/test/data/bzipdir1.bz2/bzipdir2.bz2/recordLossblockHeaderEndsAt136500.txt.bz2
------------------------------------------------------------------------------
  Merged /pig/trunk/test/org/apache/pig/test/data/bzipdir1.bz2/bzipdir2.bz2/recordLossblockHeaderEndsAt136500.txt.bz2:r1525017-1525169