You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/05/30 18:54:05 UTC

svn commit: r661790 - in /incubator/pig/branches/types: src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/backend/local/executionengine/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/mapReduceLayer/ test/org/apache/p...

Author: gates
Date: Fri May 30 09:54:04 2008
New Revision: 661790

URL: http://svn.apache.org/viewvc?rev=661790&view=rev
Log:
Changed usage of OperatorKey.scope in all operators to preserve original scope, so that operators can be associated throughout job.


Modified:
    incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
    incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld

Modified: incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/backend/hadoop/executionengine/HExecutionEngine.java Fri May 30 09:54:04 2008
@@ -216,13 +216,6 @@
 
     public PhysicalPlan compile(LogicalPlan plan,
                                 Properties properties) throws ExecException {
-        /*
-        return compile(new LogicalPlan[] { plan }, properties);
-    }
-
-    public PhysicalPlan compile(LogicalPlan[] plans,
-                                Properties properties) throws ExecException {
-                                */
         if (plan == null) {
             throw new ExecException("No Plan to compile");
         }
@@ -236,35 +229,6 @@
         } catch (VisitorException ve) {
             throw new ExecException(ve);
         }
-
-        /*
-        OperatorKey physicalKey = null;
-        for (int i = 0; i < plans.length; ++i) {
-            LogicalPlan curPlan = null;
-
-            curPlan = plans[ i ];
-     
-            OperatorKey logicalKey = curPlan.getRoot();
-            
-            physicalKey = logicalToPhysicalKeys.get(logicalKey);
-            
-            if (physicalKey == null) {
-                try {
-                physicalKey = new MapreducePlanCompiler(pigContext).
-                                        compile(curPlan.getRoot(),
-                                                curPlan.getOpTable(),
-                                                this);
-                }
-                catch (IOException e) {
-                    throw new ExecException("Failed to compile plan (" + i + ") " + logicalKey,
-                                                       e);
-                }
-                
-                logicalToPhysicalKeys.put(logicalKey, physicalKey);
-            }            
-        }
-        
-        return new MapRedPhysicalPlan(physicalKey, physicalOpTable);*/
     }
 
     public ExecJob execute(PhysicalPlan plan,
@@ -273,8 +237,9 @@
             PhysicalOperator leaf = (PhysicalOperator)plan.getLeaves().get(0);
             FileSpec spec = null;
             if(!(leaf instanceof POStore)){
-                POStore str = new POStore(new OperatorKey("HExecEngine",
-                    NodeIdGenerator.getGenerator().getNextNodeId("HExecEngine")));
+                String scope = leaf.getOperatorKey().getScope();
+                POStore str = new POStore(new OperatorKey(scope,
+                    NodeIdGenerator.getGenerator().getNextNodeId(scope)));
                 str.setPc(pigContext);
                 spec = new FileSpec(FileLocalizer.getTemporaryPath(null,
                     pigContext).toString(),
@@ -297,45 +262,6 @@
             else throw new ExecException(e.getMessage(), e);
         }
 
-        // TODO FIX Need to uncomment this with the right logic
-        /*POMapreduce pom = (POMapreduce) physicalOpTable.get(plan.getRoot());
-
-        MapReduceLauncher.initQueryStatus(pom.numMRJobs());  // initialize status, for bookkeeping purposes.
-        MapReduceLauncher.setConf(this.conf.getConfiguration());
-        MapReduceLauncher.setExecEngine(this);
-        
-        // if the final operator is a MapReduce with no output file, then send to a temp
-        // file.
-        if (pom.outputFileSpec==null) {
-            try {
-                pom.outputFileSpec = new FileSpec(FileLocalizer.getTemporaryPath(null, pigContext).toString(),
-                                                  BinStorage.class.getName());
-            }
-            catch (IOException e) {
-                throw new ExecException("Failed to obtain temp file for " + plan.getRoot().toString(), e);
-            }
-        }
-
-        try {
-            pom.open();
-            
-            Tuple t;
-            while ((t = (Tuple) pom.getNext()) != null) {
-                ;
-            }
-            
-            pom.close();
-            
-            this.materializedResults.put(pom.sourceLogicalKey,
-                                         new MapRedResult(pom.outputFileSpec,
-                                                           pom.reduceParallelism));
-        }
-        catch (IOException e) {
-            throw new ExecException(e);
-        }
-        
-        return new HJob(JOB_STATUS.COMPLETED, pigContext, pom.outputFileSpec);*/
-
     }
 
     public ExecJob submit(PhysicalPlan plan,

Modified: incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/backend/local/executionengine/LocalExecutionEngine.java Fri May 30 09:54:04 2008
@@ -108,17 +108,6 @@
     
     public PhysicalPlan compile(LogicalPlan plan,
                                 Properties properties) throws ExecException {
-        /*
-        if (plan == null) {
-            throw new ExecException("No Plan to compile");
-        }
-
-        return compile(new LogicalPlan[]{ plan } , properties);
-    }
-
-    public PhysicalPlan compile(LogicalPlan[] plans,
-                                Properties properties) throws ExecException {
-                                */
         if (plan == null) {
             throw new ExecException("No Plan to compile");
         }
@@ -132,29 +121,6 @@
         } catch (VisitorException ve) {
             throw new ExecException(ve);
         }
-
-        /*
-        OperatorKey physicalKey = null;
-        for (int i = 0; i < plans.length; ++i) {
-            LogicalPlan curPlan = null;
-
-            curPlan = plans[ i ];
-     
-            OperatorKey logicalKey = curPlan.getRoot();
-            
-            physicalKey = logicalToPhysicalKeys.get(logicalKey);
-            
-            if (physicalKey == null) {
-                physicalKey = doCompile(curPlan.getRoot(),
-                                        curPlan.getOpTable(),
-                                        properties);
-                
-                logicalToPhysicalKeys.put(logicalKey, physicalKey);
-            }
-        }
-        
-        return new LocalPhysicalPlan(physicalKey, physicalOpTable);
-        */
     }
 
     public ExecJob execute(PhysicalPlan plan,
@@ -163,8 +129,9 @@
             PhysicalOperator leaf = (PhysicalOperator)plan.getLeaves().get(0);
             FileSpec spec = null;
             if(!(leaf instanceof POStore)){
-                POStore str = new POStore(new OperatorKey("HExecEngine",
-                    NodeIdGenerator.getGenerator().getNextNodeId("HExecEngine")));
+                String scope = leaf.getOperatorKey().getScope();
+                POStore str = new POStore(new OperatorKey(scope,
+                    NodeIdGenerator.getGenerator().getNextNodeId(scope)));
                 str.setPc(pigContext);
                 spec = new FileSpec(FileLocalizer.getTemporaryPath(null,
                     pigContext).toString(),
@@ -184,29 +151,6 @@
             if (e instanceof ExecException) throw (ExecException)e;
             else throw new ExecException(e.getMessage(), e);
         }
-
-
-        // TODO Fix connect to local job runner
-        /*
-        DataBag results = BagFactory.getInstance().newDefaultBag();
-        try {
-            PhysicalOperator pp = (PhysicalOperator)physicalOpTable.get(plan.getRoot());
-
-            pp.open();
-            
-            Tuple t;
-            while ((t = (Tuple) pp.getNext()) != null) {
-                results.add(t);
-            }
-            
-            pp.close();
-        }
-        catch (IOException e) {
-            throw new ExecException(e);
-        }
-        
-        return new LocalJob(results, JOB_STATUS.COMPLETED);
-        */
     }
 
     public LocalJob submit(PhysicalPlan plan,
@@ -235,125 +179,9 @@
                                   Properties properties) 
             throws ExecException {
         
-        // TODO FIX
-        /*
-        LocalResult materializedResult = materializedResults.get(logicalKey);
-        
-        if (materializedResult != null) {
-            ExecPhysicalOperator pp = new POLoad(logicalKey.getScope(),
-                                             nodeIdGenerator.getNextNodeId(logicalKey.getScope()),
-                                             physicalOpTable,
-                                             pigContext, 
-                                             materializedResult.outFileSpec,
-                                             LogicalOperator.FIXED);
-            
-            OperatorKey ppKey = new OperatorKey(pp.getScope(), pp.getId());
-            
-            return ppKey;
-        }
-
-        OperatorKey physicalKey = new OperatorKey();
-        
-        if (compileOperator(logicalKey, logicalOpTable, properties, physicalKey)) {
-            for (int i = 0; i < logicalOpTable.get(logicalKey).getInputs().size(); ++i) {
-                ((PhysicalOperator)physicalOpTable.get(physicalKey)).inputs[i] = 
-                    doCompile(logicalOpTable.get(logicalKey).getInputs().get(i), logicalOpTable, properties);
-            }
-        }
-
-        return physicalKey;
-        */
         return null;
     }
     
-    private boolean compileOperator(OperatorKey logicalKey, 
-                                    Map<OperatorKey, LogicalOperator> logicalOpTable,
-                                    Properties properties,
-                                    OperatorKey physicalKey) 
-            throws ExecException {
-                // TODO FIX
-                /* 
-        ExecPhysicalOperator pp;
-        LogicalOperator lo = logicalOpTable.get(logicalKey);
-        String scope = lo.getOperatorKey().getScope();
-        boolean compileInputs = true;
-        
-        if (lo instanceof LOEval) {
-            
-            pp = new POEval(scope,
-                           nodeIdGenerator.getNextNodeId(scope),
-                           physicalOpTable,
-                           ((LOEval) lo).getSpec(),
-                           lo.getOutputType());
-        } 
-        else if (lo instanceof LOCogroup) {
-            pp = new POCogroup(scope,
-                               nodeIdGenerator.getNextNodeId(scope),
-                               physicalOpTable,
-                               ((LOCogroup) lo).getSpecs(),
-                               lo.getOutputType());
-        }  
-        else if (lo instanceof LOLoad) {
-            pp = new POLoad(scope,
-                            nodeIdGenerator.getNextNodeId(scope),
-                            physicalOpTable,
-                            pigContext, 
-                            ((LOLoad)lo).getInputFileSpec(),
-                            lo.getOutputType());
-        }
-        else if (lo instanceof LOSplitOutput) {
-            LOSplitOutput loso = (LOSplitOutput)lo;
-            LOSplit los = ((LOSplit)(logicalOpTable.get(loso.getInputs().get(0))));
-            
-            pp = new POSplit(scope,
-                             nodeIdGenerator.getNextNodeId(scope),
-                             physicalOpTable,
-                             doCompile(los.getInputs().get(0),
-                                       logicalOpTable,
-                                       properties), 
-                             los.getConditions(),
-                             loso.getReadFrom(),
-                             lo.getOutputType());
-            
-            compileInputs = false;
-        }
-        else if (lo instanceof LOStore) {
-            pp = new POStore(scope,
-                             nodeIdGenerator.getNextNodeId(scope),
-                             physicalOpTable,
-                             lo.getInputs().get(0),
-                             materializedResults,
-                             ((LOStore)lo).getOutputFileSpec(),
-                             ((LOStore)lo).isAppend(),
-                             pigContext);
-        } 
-        else if (lo instanceof LOUnion) {
-            pp = new POUnion(scope,
-                             nodeIdGenerator.getNextNodeId(scope),
-                             physicalOpTable,
-                             ((LOUnion)lo).getInputs().size(),
-                             lo.getOutputType());
-        } 
-        else if (lo instanceof LOSort) {
-            pp = new POSort(scope,
-                            nodeIdGenerator.getNextNodeId(scope),
-                            physicalOpTable,
-                            ((LOSort)lo).getSortSpec(),
-                            lo.getOutputType());
-        }
-        else {
-            throw new ExecException("Internal error: Unknown logical operator.");
-        }
-        
-        physicalKey.scope = pp.getScope();
-        physicalKey.id = pp.getId();
-        
-        logicalToPhysicalKeys.put(logicalKey, physicalKey);
-        
-        return compileInputs;
-        */
-            return false;
-    }
 }
 
 

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LOPrinter.java Fri May 30 09:54:04 2008
@@ -161,7 +161,7 @@
         print(mlu);
         mStream.print("(");
         mlu.getMap().visit(this);
-        mStream.print(")# " + mlu.getKey());
+        mStream.print(")# " + mlu.getOperatorKey());
     }
 
     public void visit(LOMod m) throws VisitorException {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java Fri May 30 09:54:04 2008
@@ -78,7 +78,7 @@
 	
 	@Override
 	public void visit(LOGreaterThan op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new GreaterThanExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -102,7 +102,7 @@
 	
 	@Override
 	public void visit(LOLesserThan op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new LessThanExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -125,7 +125,7 @@
 	
 	@Override
 	public void visit(LOGreaterThanEqual op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new GTOrEqualToExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -147,7 +147,7 @@
 	
 	@Override
 	public void visit(LOLesserThanEqual op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new LTOrEqualToExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -169,7 +169,7 @@
 	
 	@Override
 	public void visit(LOEqual op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new EqualToExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -191,7 +191,7 @@
 	
 	@Override
 	public void visit(LONotEqual op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new NotEqualToExpr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getLhsOperand().getType());
 		LogicalPlan lp = op.mPlan;
@@ -213,7 +213,7 @@
 	
 	@Override
 	public void visit(LOAdd op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new Add(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		LogicalPlan lp = op.mPlan;
@@ -235,7 +235,7 @@
 	
 	@Override
 	public void visit(LOSubtract op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new Subtract(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		LogicalPlan lp = op.mPlan;
@@ -257,7 +257,7 @@
 	
 	@Override
 	public void visit(LOMultiply op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new Multiply(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		LogicalPlan lp = op.mPlan;
@@ -279,7 +279,7 @@
 	
 	@Override
 	public void visit(LODivide op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new Divide(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		LogicalPlan lp = op.mPlan;
@@ -301,7 +301,7 @@
 	
 	@Override
 	public void visit(LOMod op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator exprOp = new Mod(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		LogicalPlan lp = op.mPlan;
@@ -324,7 +324,7 @@
 	@Override
 	public void visit(LOCogroup cg) throws VisitorException {
 		boolean currentPhysicalPlan = false;
-		String scope = cg.getKey().scope;
+		String scope = cg.getOperatorKey().scope;
 		List<LogicalOperator> inputs = cg.getInputs();
 		
 		POGlobalRearrange poGlobal = new POGlobalRearrange(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), cg.getRequestedParallelism());
@@ -386,7 +386,7 @@
 
 	@Override
 	public void visit(LOFilter filter) throws VisitorException {
-		String scope = filter.getKey().scope;
+		String scope = filter.getOperatorKey().scope;
 		POFilter poFilter = new POFilter(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), filter.getRequestedParallelism());
 		poFilter.setResultType(filter.getType());
 		currentPlan.add(poFilter);
@@ -415,7 +415,7 @@
 	
 	@Override
 	public void visit(LOProject op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		POProject exprOp = new POProject(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		exprOp.setResultType(op.getType());
 		exprOp.setColumn(op.getCol());
@@ -442,7 +442,7 @@
 
 	@Override
 	public void visit(LOForEach forEach) throws VisitorException {
-		String scope = forEach.getKey().scope;
+		String scope = forEach.getOperatorKey().scope;
 		//This needs to be handled specially.
 		//We need to be able to handle arbitrary levels of nesting
 		
@@ -479,7 +479,7 @@
 	@Override
 	public void visit(LOGenerate g) throws VisitorException {
 		boolean currentPhysicalPlan = false;
-		String scope = g.getKey().scope;
+		String scope = g.getOperatorKey().scope;
 		List<ExprPlan> exprPlans = new ArrayList<ExprPlan>();
 		List<LogicalPlan> plans = g.getGeneratePlans();
 		
@@ -518,7 +518,7 @@
 
 	@Override
 	public void visit(LOSort s) throws VisitorException {
-		String scope = s.getKey().scope;
+		String scope = s.getOperatorKey().scope;
 		List<LogicalPlan> logPlans = s.getSortColPlans();
 		List<ExprPlan> sortPlans = new ArrayList<ExprPlan>(logPlans.size());
 		
@@ -560,7 +560,7 @@
 	
 	@Override
 	public void visit(LODistinct op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		//This is simpler. No plans associated with this. Just create the physical operator,
 		//push it in the current plan and make the connections
 		PhysicalOperator physOp = new PODistinct(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
@@ -614,7 +614,7 @@
 	
 	@Override
 	public void visit(LOSplit split) throws VisitorException {
-		String scope = split.getKey().scope;
+		String scope = split.getOperatorKey().scope;
 		PhysicalOperator physOp = new POSplit(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), split.getRequestedParallelism());
 		LogToPhyMap.put(split, physOp);
 		
@@ -629,7 +629,7 @@
 	
 	@Override
 	public void visit(LOSplitOutput split) throws VisitorException {
-		String scope = split.getKey().scope;
+		String scope = split.getOperatorKey().scope;
 		PhysicalOperator physOp = new POFilter(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), split.getRequestedParallelism());
 		LogToPhyMap.put(split, physOp);
 		
@@ -655,7 +655,7 @@
 	
 	@Override
 	public void visit(LOUserFunc func) throws VisitorException {
-		String scope = func.getKey().scope;
+		String scope = func.getOperatorKey().scope;
 		Object f = PigContext.instantiateFuncFromSpec(func.getFuncName());
 		PhysicalOperator p;
 		if(f instanceof EvalFunc) { 
@@ -680,7 +680,7 @@
 	
 	@Override
 	public void visit(LOLoad loLoad) throws VisitorException {
-		String scope = loLoad.getKey().scope;
+		String scope = loLoad.getOperatorKey().scope;
 		//This would be a root operator. We don't need to worry about finding its predecessors
 		POLoad load = new POLoad(new OperatorKey(scope, nodeGen.getNextNodeId(scope)));
 		load.setLFile(loLoad.getInputFile());
@@ -692,7 +692,7 @@
 	
 	@Override
 	public void visit(LOStore loStore) throws VisitorException {
-		String scope = loStore.getKey().scope;
+		String scope = loStore.getOperatorKey().scope;
 		POStore store = new POStore(new OperatorKey(scope, nodeGen.getNextNodeId(scope)));
 		store.setSFile(loStore.getOutputFile());
 		store.setPc(pc);
@@ -708,7 +708,7 @@
 	
 	@Override
 	public void visit(LOConst op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ConstantExpression ce = new ConstantExpression(new OperatorKey(scope, nodeGen.getNextNodeId(scope)));
 		ce.setValue(op.getValue());
 		ce.setResultType(op.getType());
@@ -719,7 +719,7 @@
 	
 	@Override
 	public void visit(LOBinCond op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator physOp = new POBinCond(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		LogToPhyMap.put(op, physOp);
 		POBinCond phy = (POBinCond)physOp;
@@ -747,7 +747,7 @@
 	
 	@Override
 	public void visit(LONegative op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator physOp = new PONegative(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism(), null);
 		currentPlan.add(physOp);
 		
@@ -764,7 +764,7 @@
 	
 	@Override
 	public void visit(LOIsNull op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator physOp = new POIsNull(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism(), null);
 		currentPlan.add(physOp);
 		
@@ -781,7 +781,7 @@
 	
 	@Override
 	public void visit(LOMapLookup op) throws VisitorException {
-		String scope = ((OperatorKey)op.getKey()).scope;
+		String scope = ((OperatorKey)op.getOperatorKey()).scope;
 		ExpressionOperator physOp = new POMapLookUp(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism(), op.getLookUpKey());
 		physOp.setResultType(op.getType());
 		currentPlan.add(physOp);
@@ -799,7 +799,7 @@
 	
 	@Override
 	public void visit(LOCast op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		ExpressionOperator physOp = new POCast(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		currentPlan.add(physOp);
 		
@@ -817,7 +817,7 @@
 	
 	@Override
 	public void visit(LOUnion op) throws VisitorException {
-		String scope = op.getKey().scope;
+		String scope = op.getOperatorKey().scope;
 		POUnion physOp = new POUnion(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
 		currentPlan.add(physOp);
 		physOp.setResultType(op.getType());

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogicalOperator.java Fri May 30 09:54:04 2008
@@ -238,8 +238,4 @@
     public LogicalPlan getPlan() {
         return mPlan ;
     }
-    
-    public OperatorKey getKey() {
-    	return mKey;
-    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java Fri May 30 09:54:04 2008
@@ -143,7 +143,7 @@
         splitsSeen = new HashMap<OperatorKey, MapReduceOper>();
         MRPlan = new MROperPlan();
         nig = NodeIdGenerator.getGenerator();
-        scope = "MRCompiler";
+        scope = plan.getRoots().get(0).getOperatorKey().getScope();
         r = new Random(1331);
         FileLocalizer.setR(r);
         udfFinderForExpr = new UDFFinderForExpr();
@@ -943,6 +943,7 @@
         return mro;
     }
     
+    /*
     public static void main(String[] args) throws PlanException, IOException, ExecException, VisitorException {
         PigContext pc = new PigContext();
         pc.connect();
@@ -960,8 +961,8 @@
         POLoad ld = comp.getLoad();
         pj.mapPlan.add(ld);
 
-        /*POSort op = new POSort(new OperatorKey("", r.nextLong()), -1, null,
-                sortPlans, mAscCols, null);*/
+        //POSort op = new POSort(new OperatorKey("", r.nextLong()), -1, null,
+        //      sortPlans, mAscCols, null);
         PODistinct op = new PODistinct(new OperatorKey("", r.nextLong()),
                 -1, null);
         pj.mapPlan.addAsLeaf(op);
@@ -975,4 +976,5 @@
         PlanPrinter<MapReduceOper, MROperPlan> pp = new PlanPrinter<MapReduceOper, MROperPlan>(plan);
         pp.print(System.out);
     }
+    */
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java Fri May 30 09:54:04 2008
@@ -82,7 +82,7 @@
         reducePlan = new PhysicalPlan<PhysicalOperator>();
         UDFs = new ArrayList<String>();
         nig = NodeIdGenerator.getGenerator();
-        scope = "MapReduceOper";
+        scope = k.getScope();
     }
 
     /*@Override

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java Fri May 30 09:54:04 2008
@@ -874,10 +874,14 @@
 
             php.explain(System.out);
             System.out.println();
-            System.out.println(compiledPlan);
+            System.out.println("<<<" + compiledPlan + ">>>");
+            System.out.println("-------------");
+            System.out.println("Golden");
+            System.out.println("<<<" + goldenPlan + ">>>");
             System.out.println("-------------");
             //System.out.println(compiledPlan.compareTo(goldenPlan)==0);
-            assertEquals(true, compiledPlan.compareTo(goldenPlan) == 0);
+            //assertEquals(true, compiledPlan.compareTo(goldenPlan) == 0);
+            assertEquals(goldenPlan, compiledPlan);
         }
     }
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld Fri May 30 09:54:04 2008
@@ -1,13 +1,13 @@
-MapReduce(-1) - MRCompiler-3:
+MapReduce(-1) - -3:
 |   Store(DummyFil:DummyLdr) - -6748240903696823165
 |   |
 |   |---Filter - -7469509242284658386
 |       |
 |       |---Package - --7212359720440714287
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-2
+|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -2
 |
-|---MapReduce(-1) - MRCompiler-0:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-1
+|---MapReduce(-1) - -0:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -1
     |   |
     |   |---Local Rearrange - -8345455294066939854
     |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld Fri May 30 09:54:04 2008
@@ -1,23 +1,23 @@
-MapReduce(-1) - MRCompiler-68:
+MapReduce(-1) - -72:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - --2325244147060806375
 |   |
 |   |---Union - --8240903279973257769
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-69
+|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -73
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-71
+|       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -75
 |
-|---MapReduce(-1) - MRCompiler-60:
-|   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-70
+|---MapReduce(-1) - -63:
+|   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -74
 |   |   |
 |   |   |---Package - --6957591191795645940
-|   |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-61
+|   |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -64
 |   |   
-|   |   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-63
+|   |   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -66
 |   |
-|   |---MapReduce(20) - MRCompiler-58:
-|   |   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-62
+|   |---MapReduce(20) - -61:
+|   |   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -65
 |   |   |   |
 |   |   |   |---Local Rearrange - --5190071275724378681
 |   |   |       |
@@ -28,8 +28,8 @@
 |   |   |       |
 |   |   |       |---Load(DummyFil:DummyLdr) - --8361563503038121624
 |   |
-|   |---MapReduce(30) - MRCompiler-59:
-|       |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-64
+|   |---MapReduce(30) - -62:
+|       |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -67
 |       |   |
 |       |   |---Local Rearrange - -580704905698185869
 |       |       |
@@ -38,13 +38,13 @@
 |       |   |
 |       |   |---Load(DummyFil:DummyLdr) - -5515425171581967372
 |
-|---MapReduce(-1) - MRCompiler-67:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-72
+|---MapReduce(-1) - -70:
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -76
     |   |
     |   |---Filter - --1194577301115518934
     |       |
     |       |---Package - --8027742474430787324
-    |   Union - MapReduceOper-3
+    |   Union - -71
     |   |
     |   |---Local Rearrange - --5768272326302808468
     |   |   |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld Fri May 30 09:54:04 2008
@@ -1,18 +1,18 @@
-MapReduce(-1) - MRCompiler-81:
+MapReduce(-1) - -87:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -7219092146411102494
 |   |
 |   |---Union - -4266652190633805129
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-82
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -88
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-84
+|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -90
 |
-|---MapReduce(-1) - MRCompiler-75:
-|   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-83
+|---MapReduce(-1) - -79:
+|   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -89
 |   |   |
 |   |   |---Package - --2530083186736419352
-|   |   Union - MapReduceOper-4
+|   |   Union - -80
 |   |   |
 |   |   |---Local Rearrange - --3008031181120208412
 |   |   |   |
@@ -20,23 +20,23 @@
 |   |   |       |
 |   |   |       |---Load(DummyFil:DummyLdr) - --5123390619301085966
 |   |   |
-|   |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-76
+|   |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -81
 |   |
-|   |---MapReduce(-1) - MRCompiler-73:
-|       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-77
+|   |---MapReduce(-1) - -77:
+|       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -82
 |       |   |
 |       |   |---Package - --4776862487779398064
 |       |   Local Rearrange - --3624738819541751735
 |       |   |
 |       |   |---Load(DummyFil:DummyLdr) - -3719468117239527682
 |
-|---MapReduce(-1) - MRCompiler-80:
-    |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-85
+|---MapReduce(-1) - -85:
+    |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -91
     |   |
     |   |---Filter - -7127277012934370361
     |       |
     |       |---Package - --1036765666284482668
-    |   Union - MapReduceOper-5
+    |   Union - -86
     |   |
     |   |---Local Rearrange - -810744320494301308
     |   |   |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld Fri May 30 09:54:04 2008
@@ -1,23 +1,23 @@
-MapReduce(-1) - MRCompiler-92:
+MapReduce(-1) - -98:
 |   Store(DummyFil:DummyLdr) - -913122466036599874
 |   |
 |   |---Package - -8515950928528386562
-|   Union - MapReduceOper-6
+|   Union - -99
 |   |
 |   |---Local Rearrange - --7092666336106039025
 |   |   |
 |   |   |---Filter - --506205335735031954
 |   |       |
-|   |       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-88
+|   |       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -94
 |   |
 |   |---Local Rearrange - -8957152010410190649
 |       |
 |       |---Filter - --2465563180022385815
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-90
+|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -96
 |
-|---MapReduce(-1) - MRCompiler-86:
+|---MapReduce(-1) - -92:
     Reduce Plan Empty
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-87
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -93
     |   |
     |   |---Load(DummyFil:DummyLdr) - --90170124379688035
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld Fri May 30 09:54:04 2008
@@ -1,4 +1,4 @@
-MapReduce(-1) - MRCompiler-105:
+MapReduce(-1) - -113:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -3777104251028634198
 |   |
@@ -6,32 +6,32 @@
 |       |
 |       |---Filter - --4160312061837144266
 |       |   |
-|       |   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-101
+|       |   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -109
 |       |
 |       |---Filter - -2997708366016271267
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-103
+|           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -111
 |
-|---MapReduce(-1) - MRCompiler-99:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-100
+|---MapReduce(-1) - -106:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -108
     |   |
     |   |---Package - --1195368474144577564
-    |   Union - MapReduceOper-7
+    |   Union - -107
     |   |
     |   |---Local Rearrange - --8706893458091286727
     |   |   |
     |   |   |---Filter - -6693653468874366867
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-95
+    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -102
     |   |
     |   |---Local Rearrange - --8512881738335808286
     |       |
     |       |---Filter - --3347320220748403847
     |           |
-    |           |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-97
+    |           |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -104
     |
-    |---MapReduce(-1) - MRCompiler-93:
+    |---MapReduce(-1) - -100:
         Reduce Plan Empty
-        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-94
+        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -101
         |   |
         |   |---Load(DummyFil:DummyLdr) - -5969405952749301492
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld Fri May 30 09:54:04 2008
@@ -1,4 +1,4 @@
-MapReduce(100) - MRCompiler-126:
+MapReduce(100) - -135:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - --7335024873119453444
 |   |
@@ -6,60 +6,60 @@
 |       |
 |       |---Filter - --4248200967728536480
 |       |   |
-|       |   |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-122
+|       |   |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -131
 |       |
 |       |---Filter - -4589138876054328603
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-124
+|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -133
 |
-|---MapReduce(40) - MRCompiler-120:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-121
+|---MapReduce(40) - -128:
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -130
     |   |
     |   |---Package - --1848504978980807369
-    |   Union - MapReduceOper-8
+    |   Union - -129
     |   |
     |   |---Local Rearrange - --5908426805312852480
     |   |   |
     |   |   |---Filter - --8479692259657755370
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-111
+    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -119
     |   |
     |   |---Local Rearrange - -5965044993061572808
     |   |   |
     |   |   |---Filter - --2247078527532208396
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-113
+    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -121
     |   |
     |   |---Local Rearrange - -8914265632748254170
     |       |
     |       |---Filter - -9193928674704944093
     |           |
-    |           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-118
+    |           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -126
     |
-    |---MapReduce(20) - MRCompiler-109:
+    |---MapReduce(20) - -117:
     |   Reduce Plan Empty
-    |   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-110
+    |   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -118
     |   |   |
     |   |   |---Filter - --2771799342014688397
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-108
+    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -116
     |   |
-    |   |---MapReduce(-1) - MRCompiler-106:
+    |   |---MapReduce(-1) - -114:
     |       Reduce Plan Empty
-    |       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-107
+    |       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -115
     |       |   |
     |       |   |---Load(DummyFil:DummyLdr) - -7955165653808522445
     |
-    |---MapReduce(10) - MRCompiler-116:
+    |---MapReduce(10) - -124:
         Reduce Plan Empty
-        |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-117
+        |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -125
         |   |
         |   |---Filter - --997494992827460962
         |       |
-        |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-115
+        |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -123
         |
-        |---MapReduce(-1) - MRCompiler-106:
+        |---MapReduce(-1) - -114:
             Reduce Plan Empty
-            |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-107
+            |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -115
             |   |
             |   |---Load(DummyFil:DummyLdr) - -7955165653808522445
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld Fri May 30 09:54:04 2008
@@ -1,4 +1,4 @@
-MapReduce(-1,GFCross) - MRCompiler-161:
+MapReduce(-1,GFCross) - -170:
 |   Store(DummyFil:DummyLdr) - -5165956429696944631
 |   |
 |   |---For Each - -2197807331204639125
@@ -10,10 +10,10 @@
 |       |   |   |---Project(*) - -5878058249424113280
 |       |
 |       |---Package - -7085374960058856883
-|   Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-160
+|   Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -169
 |
-|---MapReduce(-1,AVG) - MRCompiler-158:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - MRCompiler-159
+|---MapReduce(-1,AVG) - -167:
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -168
     |   |
     |   |---Local Rearrange - -7398260302074824818
     |       |
@@ -26,10 +26,10 @@
     |           |   |   |---Project(*) - -2082992246427879202
     |           |
     |           |---Package - -737965411848544085
-    |   Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-157
+    |   Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -166
     |
-    |---MapReduce(20,FindQuantiles,SUM) - MRCompiler-149:
-        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - MRCompiler-156
+    |---MapReduce(20,FindQuantiles,SUM) - -158:
+        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -165
         |   |
         |   |---Local Rearrange - --3359714221559579809
         |       |
@@ -45,60 +45,60 @@
         |           |   |   |
         |           |   |   |---Project(*) - --8002381389674382470
         |           |
-        |           |---For Each - MRCompiler-155
+        |           |---For Each - -164
         |               |   |
-        |               |   POGenerate(true)  - MRCompiler-154
+        |               |   POGenerate(true)  - -163
         |               |   |   |
-        |               |   |   Project(1) - MRCompiler-153
+        |               |   |   Project(1) - -162
         |               |
-        |               |---Package - MRCompiler-152
-        |   Local Rearrange - MRCompiler-151
+        |               |---Package - -161
+        |   Local Rearrange - -160
         |   |   |
-        |   |   Project(0) - MRCompiler-150
+        |   |   Project(0) - -159
         |   |
-        |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-148
+        |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -157
         |
-        |---MapReduce(-1,FindQuantiles,TestMRCompiler$WeirdComparator) - MRCompiler-130:
-            |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-147
+        |---MapReduce(-1,FindQuantiles,TestMRCompiler$WeirdComparator) - -139:
+            |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -156
             |   |
-            |   |---For Each - MRCompiler-146
+            |   |---For Each - -155
             |       |   |
-            |       |   POGenerate(false)  - MRCompiler-145
+            |       |   POGenerate(false)  - -154
             |       |   |   |
-            |       |   |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles) - MRCompiler-144
+            |       |   |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles) - -153
             |       |   |   |
-            |       |   |   |---Project(*) - MRCompiler-143
+            |       |   |   |---Project(*) - -152
             |       |
-            |       |---For Each - MRCompiler-142
+            |       |---For Each - -151
             |           |   |
-            |           |   POGenerate(false,false)  - MRCompiler-141
+            |           |   POGenerate(false,false)  - -150
             |           |   |   |
-            |           |   |   Constant(20) - MRCompiler-140
+            |           |   |   Constant(20) - -149
             |           |   |   |
-            |           |   |   Project(*) - MRCompiler-139
+            |           |   |   Project(*) - -148
             |           |   |
             |           |   |---POSort - --776319888013965510
             |           |       |   |
-            |           |       |   Project(*) - MRCompiler-138
+            |           |       |   Project(*) - -147
             |           |       |
-            |           |       |---Project(1) - MRCompiler-137
+            |           |       |---Project(1) - -146
             |           |
-            |           |---Package - MRCompiler-136
-            |   Local Rearrange - MRCompiler-135
+            |           |---Package - -145
+            |   Local Rearrange - -144
             |   |   |
-            |   |   Constant(all) - MRCompiler-134
+            |   |   Constant(all) - -143
             |   |
-            |   |---For Each - MRCompiler-133
+            |   |---For Each - -142
             |       |   |
-            |       |   POGenerate(true)  - MRCompiler-132
+            |       |   POGenerate(true)  - -141
             |       |   |   |
-            |       |   |   Project(0) - MRCompiler-131
+            |       |   |   Project(0) - -140
             |       |
-            |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.impl.builtin.RandomSampleLoader) - MRCompiler-129
+            |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.impl.builtin.RandomSampleLoader) - -138
             |
-            |---MapReduce(-1) - MRCompiler-127:
+            |---MapReduce(-1) - -136:
                 Reduce Plan Empty
-                |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-128
+                |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -137
                 |   |
                 |   |---Filter - --4867358672373443663
                 |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld Fri May 30 09:54:04 2008
@@ -1,40 +1,40 @@
-MapReduce(-1) - MRCompiler-176:
+MapReduce(-1) - -185:
 |   Store(DummyFil:DummyLdr) - -6079615556647418436
 |   |
-|   |---For Each - MRCompiler-180
+|   |---For Each - -189
 |       |   |
-|       |   POGenerate(false)  - MRCompiler-179
+|       |   POGenerate(false)  - -188
 |       |   |   |
-|       |   |   Project(0) - MRCompiler-178
+|       |   |   Project(0) - -187
 |       |
-|       |---Package - MRCompiler-177
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-175
+|       |---Package - -186
+|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -184
 |
-|---MapReduce(-1) - MRCompiler-171:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-174
+|---MapReduce(-1) - -180:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -183
     |   |
-    |   |---Local Rearrange - MRCompiler-173
+    |   |---Local Rearrange - -182
     |       |   |
-    |       |   Project(*) - MRCompiler-172
+    |       |   Project(*) - -181
     |       |
     |       |---Package - -8219725798912083822
-    |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-170
+    |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -179
     |
-    |---MapReduce(-1) - MRCompiler-162:
-        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-169
+    |---MapReduce(-1) - -171:
+        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -178
         |   |
         |   |---Local Rearrange - --3889827013424534115
         |       |
-        |       |---For Each - MRCompiler-168
+        |       |---For Each - -177
         |           |   |
-        |           |   POGenerate(false)  - MRCompiler-167
+        |           |   POGenerate(false)  - -176
         |           |   |   |
-        |           |   |   Project(0) - MRCompiler-166
+        |           |   |   Project(0) - -175
         |           |
-        |           |---Package - MRCompiler-165
-        |   Local Rearrange - MRCompiler-164
+        |           |---Package - -174
+        |   Local Rearrange - -173
         |   |   |
-        |   |   Project(*) - MRCompiler-163
+        |   |   Project(*) - -172
         |   |
         |   |---Filter - --1613182091613226659
         |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld Fri May 30 09:54:04 2008
@@ -1,23 +1,23 @@
-MapReduce(-1) - MRCompiler-6:
+MapReduce(-1) - -6:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -8637487025682524492
 |   |
 |   |---Union - --2655303127943013956
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-7
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -7
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-9
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -9
 |
-|---MapReduce(-1) - MRCompiler-4:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-8
+|---MapReduce(-1) - -4:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -8
 |   |   |
 |   |   |---Package - --8216215966586363937
 |   |   Local Rearrange - -5679595123645092366
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - -990040854696137546
 |
-|---MapReduce(-1) - MRCompiler-5:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-10
+|---MapReduce(-1) - -5:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -10
     |   |
     |   |---Package - -7473175511145418837
     |   Local Rearrange - --3833933141637499382

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld Fri May 30 09:54:04 2008
@@ -1,27 +1,27 @@
-MapReduce(-1) - MRCompiler-14:
+MapReduce(-1) - -14:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -5879589559059467399
 |   |
 |   |---Union - -6108697957385222190
 |       |
-|       |---Filter - -689137294940608050
-|       |   |
-|       |   |---Load(DummyFil:DummyLdr) - --8723304958939002625
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -15
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-15
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -17
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-17
+|       |---Filter - -689137294940608050
+|           |
+|           |---Load(DummyFil:DummyLdr) - --8723304958939002625
 |
-|---MapReduce(-1) - MRCompiler-11:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-16
+|---MapReduce(-1) - -11:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -16
 |   |   |
 |   |   |---Package - --2449990780759860228
 |   |   Local Rearrange - --171729478481529687
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - -3527883492192621891
 |
-|---MapReduce(-1) - MRCompiler-12:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-18
+|---MapReduce(-1) - -12:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -18
     |   |
     |   |---Package - --961974426096759446
     |   Local Rearrange - -3720949273928245639

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld Fri May 30 09:54:04 2008
@@ -1,31 +1,31 @@
-MapReduce(-1) - MRCompiler-23:
+MapReduce(-1) - -23:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -956528893337238225
 |   |
 |   |---Union - -3055962796670893316
 |       |
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -24
+|       |
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -26
+|       |
 |       |---Filter - -2288440041178982968
 |       |   |
 |       |   |---Load(DummyFil:DummyLdr) - -913150185705910016
 |       |
 |       |---Filter - -4024598850351165272
-|       |   |
-|       |   |---Load(DummyFil:DummyLdr) - --5840807574562740596
-|       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-24
-|       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-26
+|           |
+|           |---Load(DummyFil:DummyLdr) - --5840807574562740596
 |
-|---MapReduce(-1) - MRCompiler-19:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-25
+|---MapReduce(-1) - -19:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -25
 |   |   |
 |   |   |---Package - --4083686173685839766
 |   |   Local Rearrange - --5733160635931065595
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - -3742910951635599848
 |
-|---MapReduce(-1) - MRCompiler-21:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-27
+|---MapReduce(-1) - -21:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -27
     |   |
     |   |---Package - -3943647700946858188
     |   Local Rearrange - -4962214768762054129

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld Fri May 30 09:54:04 2008
@@ -1,4 +1,4 @@
-MapReduce(-1) - MRCompiler-30:
+MapReduce(-1) - -30:
 Reduce Plan Empty
 |   Store(DummyFil:DummyLdr) - -340188275237520979
 |   |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld Fri May 30 09:54:04 2008
@@ -1,13 +1,13 @@
-MapReduce(-1) - MRCompiler-33:
+MapReduce(-1) - -33:
 |   Store(DummyFil:DummyLdr) - -8531323295221133204
 |   |
 |   |---Package - -3515731110955624708
-|   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-34
+|   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -34
 |   
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-36
+|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -36
 |
-|---MapReduce(-1) - MRCompiler-31:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-35
+|---MapReduce(-1) - -31:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -35
 |   |   |
 |   |   |---Local Rearrange - --7797378030368227515
 |   |       |
@@ -16,8 +16,8 @@
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - --5941754500044119766
 |
-|---MapReduce(-1) - MRCompiler-32:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-37
+|---MapReduce(-1) - -32:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -37
     |   |
     |   |---Local Rearrange - -6869916393445222618
     |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld Fri May 30 09:54:04 2008
@@ -1,19 +1,19 @@
-MapReduce(-1) - MRCompiler-41:
+MapReduce(-1) - -41:
 |   Store(DummyFil:DummyLdr) - -8205341881606740757
 |   |
 |   |---Package - -8067897495211048679
-|   Union - MapReduceOper-0
+|   Union - -42
 |   |
-|   |---Filter - -8816201798088731301
-|   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -6411189422016119355
+|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -43
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-42
+|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -45
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-44
+|   |---Filter - -8816201798088731301
+|       |
+|       |---Load(DummyFil:DummyLdr) - -6411189422016119355
 |
-|---MapReduce(-1) - MRCompiler-38:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-43
+|---MapReduce(-1) - -38:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -44
 |   |   |
 |   |   |---Local Rearrange - --8754957318949788629
 |   |       |
@@ -22,8 +22,8 @@
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - --1607475648664293401
 |
-|---MapReduce(-1) - MRCompiler-39:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-45
+|---MapReduce(-1) - -39:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -46
     |   |
     |   |---Local Rearrange - -2056844330428347996
     |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld Fri May 30 09:54:04 2008
@@ -1,23 +1,23 @@
-MapReduce(-1) - MRCompiler-50:
+MapReduce(-1) - -51:
 |   Store(DummyFil:DummyLdr) - -370083002387034672
 |   |
 |   |---Package - --7096529877081178520
-|   Union - MapReduceOper-1
+|   Union - -52
 |   |
 |   |---Filter - --7868505214447593853
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - --5177745552827005198
 |   |
-|   |---Filter - -6305091296204163466
-|   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -1236980712440527800
+|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -53
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-51
+|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -55
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-53
+|   |---Filter - -6305091296204163466
+|       |
+|       |---Load(DummyFil:DummyLdr) - -1236980712440527800
 |
-|---MapReduce(-1) - MRCompiler-48:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - MRCompiler-52
+|---MapReduce(-1) - -49:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -54
 |   |   |
 |   |   |---Local Rearrange - -6790164331631653200
 |   |       |
@@ -26,8 +26,8 @@
 |   |   |
 |   |   |---Load(DummyFil:DummyLdr) - -6620645493024302760
 |
-|---MapReduce(-1) - MRCompiler-49:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - MRCompiler-54
+|---MapReduce(-1) - -50:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -56
     |   |
     |   |---Local Rearrange - -8729990799028586395
     |       |

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld?rev=661790&r1=661789&r2=661790&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld Fri May 30 09:54:04 2008
@@ -1,8 +1,8 @@
-MapReduce(-1) - MRCompiler-57:
+MapReduce(-1) - -59:
 |   Store(DummyFil:DummyLdr) - -7506868571066332964
 |   |
 |   |---Package - -7034055442337723447
-|   Union - MapReduceOper-2
+|   Union - -60
 |   |
 |   |---Filter - -2468262982799101733
 |   |   |